diff --git a/.github/workflows/pr_build.yml b/.github/workflows/pr_build.yml
index 981905ec7..ce81ea2b2 100644
--- a/.github/workflows/pr_build.yml
+++ b/.github/workflows/pr_build.yml
@@ -46,7 +46,7 @@ jobs:
os: [ubuntu-latest]
java_version: [8, 11, 17]
test-target: [rust, java]
- spark-version: ['3.4']
+ spark-version: ['3.5']
scala-version: ['2.12', '2.13']
is_push_event:
- ${{ github.event_name == 'push' }}
@@ -109,7 +109,7 @@ jobs:
os: [ubuntu-latest]
java_version: [8, 11, 17]
test-target: [java]
- spark-version: ['3.3']
+ spark-version: ['3.3', '3.4']
scala-version: ['2.12', '2.13']
fail-fast: false
name: ${{ matrix.os }}/java ${{ matrix.java_version }}-spark-${{matrix.spark-version}}-scala-${{matrix.scala-version}}/${{ matrix.test-target }}
@@ -134,7 +134,7 @@ jobs:
os: [macos-13]
java_version: [8, 11, 17]
test-target: [rust, java]
- spark-version: ['3.4']
+ spark-version: ['3.4', '3.5']
scala-version: ['2.12', '2.13']
fail-fast: false
if: github.event_name == 'push'
@@ -161,7 +161,7 @@ jobs:
matrix:
java_version: [8, 11, 17]
test-target: [rust, java]
- spark-version: ['3.4']
+ spark-version: ['3.4', '3.5']
scala-version: ['2.12', '2.13']
is_push_event:
- ${{ github.event_name == 'push' }}
@@ -247,7 +247,7 @@ jobs:
matrix:
java_version: [8, 17]
test-target: [java]
- spark-version: ['3.3']
+ spark-version: ['3.3', '3.4']
scala-version: ['2.12', '2.13']
exclude:
- java_version: 8
diff --git a/common/pom.xml b/common/pom.xml
index cc1f44481..b59d7b187 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -181,6 +181,7 @@ under the License.
+
diff --git a/common/src/main/spark-3.x/org/apache/comet/shims/ShimBatchReader.scala b/common/src/main/spark-3.3/org/apache/comet/shims/ShimBatchReader.scala
similarity index 54%
rename from common/src/main/spark-3.x/org/apache/comet/shims/ShimBatchReader.scala
rename to common/src/main/spark-3.3/org/apache/comet/shims/ShimBatchReader.scala
index 18f91acc3..3cbca896f 100644
--- a/common/src/main/spark-3.x/org/apache/comet/shims/ShimBatchReader.scala
+++ b/common/src/main/spark-3.3/org/apache/comet/shims/ShimBatchReader.scala
@@ -24,31 +24,13 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile
object ShimBatchReader {
- // TODO: remove after dropping Spark 3.3 support and directly call PartitionedFile
def newPartitionedFile(partitionValues: InternalRow, file: String): PartitionedFile =
- classOf[PartitionedFile].getDeclaredConstructors
- .map(c =>
- c.getParameterCount match {
- case 5 =>
- c.newInstance(
- partitionValues,
- file,
- Long.box(-1), // -1 means we read the entire file
- Long.box(-1),
- Array.empty[String])
- case 7 =>
- c.newInstance(
- partitionValues,
- c.getParameterTypes()(1)
- .getConstructor(classOf[String])
- .newInstance(file)
- .asInstanceOf[AnyRef],
- Long.box(-1), // -1 means we read the entire file
- Long.box(-1),
- Array.empty[String],
- Long.box(0),
- Long.box(0))
- })
- .head
- .asInstanceOf[PartitionedFile]
+ PartitionedFile(
+ partitionValues,
+ file,
+ -1, // -1 means we read the entire file
+ -1,
+ Array.empty[String],
+ 0,
+ 0)
}
diff --git a/common/src/main/spark-3.4/org/apache/comet/shims/ShimBatchReader.scala b/common/src/main/spark-3.4/org/apache/comet/shims/ShimBatchReader.scala
new file mode 100644
index 000000000..17b60e0e5
--- /dev/null
+++ b/common/src/main/spark-3.4/org/apache/comet/shims/ShimBatchReader.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.comet.shims
+
+import org.apache.spark.paths.SparkPath
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+
+object ShimBatchReader {
+
+ def newPartitionedFile(partitionValues: InternalRow, file: String): PartitionedFile =
+ PartitionedFile(
+ partitionValues,
+ SparkPath.fromPathString(file),
+ -1, // -1 means we read the entire file
+ -1,
+ Array.empty[String],
+ 0,
+ 0)
+}
diff --git a/common/src/main/spark-3.5/org/apache/comet/shims/ShimBatchReader.scala b/common/src/main/spark-3.5/org/apache/comet/shims/ShimBatchReader.scala
new file mode 100644
index 000000000..ec11caf89
--- /dev/null
+++ b/common/src/main/spark-3.5/org/apache/comet/shims/ShimBatchReader.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.comet.shims
+
+import org.apache.spark.paths.SparkPath
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+
+object ShimBatchReader {
+
+ def newPartitionedFile(partitionValues: InternalRow, file: String): PartitionedFile =
+ PartitionedFile(
+ partitionValues,
+ SparkPath.fromPathString(file),
+ -1, // -1 means we read the entire file
+ -1,
+ Array.empty[String],
+ 0,
+ 0,
+ Map.empty)
+}
diff --git a/docs/source/user-guide/overview.md b/docs/source/user-guide/overview.md
index 87f5f286c..d7ecf04d3 100644
--- a/docs/source/user-guide/overview.md
+++ b/docs/source/user-guide/overview.md
@@ -40,7 +40,19 @@ The following diagram illustrates the architecture of Comet:
## Current Status
-The project is currently integrated into Apache Spark 3.3, and 3.4.
+Comet currently supports the following versions of Apache Spark:
+
+- 3.3.x
+- 3.4.x
+
+Experimental support is provided for the following versions of Apache Spark and is intended for development/testing
+use only and should not be used in production yet.
+
+- 3.5.x
+- 4.0.0-preview1
+
+Note that Comet may not fully work with proprietary forks of Apache Spark such as the Spark versions offered by
+Cloud Service Providers.
## Feature Parity with Apache Spark
diff --git a/pom.xml b/pom.xml
index 16ca60a3b..4dcc75834 100644
--- a/pom.xml
+++ b/pom.xml
@@ -91,8 +91,11 @@ under the License.
-ea -Xmx4g -Xss4m ${extraJavaTestArgs}
spark-3.3-plus
spark-3.4-plus
+ not-needed
+ spark-pre-3.5
spark-3.x
spark-3.4
+ spark-pre-3.5
@@ -547,6 +550,21 @@ under the License.
+
+
+ spark-3.5
+
+ 2.12.18
+ 3.5.1
+ 3.5
+ 1.13.1
+ spark-3.5
+ not-needed
+ not-needed
+ spark-3.5
+
+
+
spark-4.0
@@ -561,6 +579,8 @@ under the License.
2.0.13
spark-4.0
not-needed-yet
+ not-needed
+ not-needed
17
${java.version}
diff --git a/spark/pom.xml b/spark/pom.xml
index 84e2e501f..6b3b36aa5 100644
--- a/spark/pom.xml
+++ b/spark/pom.xml
@@ -252,6 +252,8 @@ under the License.
+
+
@@ -267,6 +269,7 @@ under the License.
+
diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
index c19395684..37ca55e27 100644
--- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
+++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
@@ -1025,6 +1025,10 @@ object CometSparkSessionExtensions extends Logging {
org.apache.spark.SPARK_VERSION >= "3.4"
}
+ def isSpark35Plus: Boolean = {
+ org.apache.spark.SPARK_VERSION >= "3.5"
+ }
+
def isSpark40Plus: Boolean = {
org.apache.spark.SPARK_VERSION >= "4.0"
}
diff --git a/spark/src/main/spark-3.5/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/CometExprShim.scala
new file mode 100644
index 000000000..7709957b4
--- /dev/null
+++ b/spark/src/main/spark-3.5/org/apache/comet/shims/CometExprShim.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.comet.shims
+
+import org.apache.comet.expressions.CometEvalMode
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.types.{DataType, TimestampNTZType}
+
+/**
+ * `CometExprShim` acts as a shim for for parsing expressions from different Spark versions.
+ */
+trait CometExprShim {
+ /**
+ * Returns a tuple of expressions for the `unhex` function.
+ */
+ protected def unhexSerde(unhex: Unhex): (Expression, Expression) = {
+ (unhex.child, Literal(unhex.failOnError))
+ }
+
+ protected def isTimestampNTZType(dt: DataType): Boolean = dt match {
+ case _: TimestampNTZType => true
+ case _ => false
+ }
+
+ protected def evalMode(c: Cast): CometEvalMode.Value =
+ CometEvalModeUtil.fromSparkEvalMode(c.evalMode)
+}
+
+object CometEvalModeUtil {
+ def fromSparkEvalMode(evalMode: EvalMode.Value): CometEvalMode.Value = evalMode match {
+ case EvalMode.LEGACY => CometEvalMode.LEGACY
+ case EvalMode.TRY => CometEvalMode.TRY
+ case EvalMode.ANSI => CometEvalMode.ANSI
+ }
+}
+
diff --git a/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSQLConf.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSQLConf.scala
new file mode 100644
index 000000000..aafe4aa7c
--- /dev/null
+++ b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSQLConf.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.comet.shims
+
+import org.apache.spark.sql.internal.LegacyBehaviorPolicy
+import org.apache.spark.sql.internal.SQLConf
+
+trait ShimSQLConf {
+
+ /**
+ * Spark 3.4 renamed parquetFilterPushDownStringStartWith to
+ * parquetFilterPushDownStringPredicate
+ */
+ protected def getPushDownStringPredicate(sqlConf: SQLConf): Boolean =
+ sqlConf.parquetFilterPushDownStringPredicate
+
+ protected val LEGACY = LegacyBehaviorPolicy.LEGACY
+ protected val CORRECTED = LegacyBehaviorPolicy.CORRECTED
+}
diff --git a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala
new file mode 100644
index 000000000..3c6f764cf
--- /dev/null
+++ b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.comet.shims
+
+import org.apache.comet.shims.ShimFileFormat
+
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil}
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions
+import org.apache.spark.sql.types.StructType
+
+trait ShimCometScanExec {
+ def wrapped: FileSourceScanExec
+
+ lazy val fileConstantMetadataColumns: Seq[AttributeReference] =
+ wrapped.fileConstantMetadataColumns
+
+ protected def newFileScanRDD(
+ fsRelation: HadoopFsRelation,
+ readFunction: PartitionedFile => Iterator[InternalRow],
+ filePartitions: Seq[FilePartition],
+ readSchema: StructType,
+ options: ParquetOptions): FileScanRDD = new FileScanRDD(
+ fsRelation.sparkSession,
+ readFunction,
+ filePartitions,
+ readSchema,
+ fileConstantMetadataColumns,
+ Map.empty,
+ options)
+
+ protected def invalidBucketFile(path: String, sparkVersion: String): Throwable =
+ new SparkException("INVALID_BUCKET_FILE", Map("path" -> path), null)
+
+ protected def isNeededForSchema(sparkSchema: StructType): Boolean = {
+ // TODO: remove after PARQUET-2161 becomes available in Parquet (tracked in SPARK-39634)
+ ShimFileFormat.findRowIndexColumnIndexInSchema(sparkSchema) >= 0
+ }
+
+ protected def getPartitionedFile(f: FileStatusWithMetadata, p: PartitionDirectory): PartitionedFile =
+ PartitionedFileUtil.getPartitionedFile(f, p.values)
+
+ protected def splitFiles(sparkSession: SparkSession,
+ file: FileStatusWithMetadata,
+ filePath: Path,
+ isSplitable: Boolean,
+ maxSplitBytes: Long,
+ partitionValues: InternalRow): Seq[PartitionedFile] =
+ PartitionedFileUtil.splitFiles(sparkSession, file, isSplitable, maxSplitBytes, partitionValues)
+}
diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimSQLConf.scala b/spark/src/main/spark-pre-3.5/org/apache/comet/shims/ShimSQLConf.scala
similarity index 100%
rename from spark/src/main/spark-3.x/org/apache/comet/shims/ShimSQLConf.scala
rename to spark/src/main/spark-pre-3.5/org/apache/comet/shims/ShimSQLConf.scala
diff --git a/spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-pre-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala
similarity index 100%
rename from spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala
rename to spark/src/main/spark-pre-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt
new file mode 100644
index 000000000..d35b9049e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt
@@ -0,0 +1,287 @@
+== Physical Plan ==
+TakeOrderedAndProject (43)
++- * Project (42)
+ +- * BroadcastHashJoin Inner BuildRight (41)
+ :- * Project (36)
+ : +- * BroadcastHashJoin Inner BuildRight (35)
+ : :- * Project (29)
+ : : +- * BroadcastHashJoin Inner BuildRight (28)
+ : : :- * Filter (13)
+ : : : +- * HashAggregate (12)
+ : : : +- Exchange (11)
+ : : : +- * ColumnarToRow (10)
+ : : : +- CometHashAggregate (9)
+ : : : +- CometProject (8)
+ : : : +- CometBroadcastHashJoin (7)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_returns (1)
+ : : : +- CometBroadcastExchange (6)
+ : : : +- CometProject (5)
+ : : : +- CometFilter (4)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : : +- BroadcastExchange (27)
+ : : +- * Filter (26)
+ : : +- * HashAggregate (25)
+ : : +- Exchange (24)
+ : : +- * HashAggregate (23)
+ : : +- * HashAggregate (22)
+ : : +- Exchange (21)
+ : : +- * ColumnarToRow (20)
+ : : +- CometHashAggregate (19)
+ : : +- CometProject (18)
+ : : +- CometBroadcastHashJoin (17)
+ : : :- CometFilter (15)
+ : : : +- CometScan parquet spark_catalog.default.store_returns (14)
+ : : +- ReusedExchange (16)
+ : +- BroadcastExchange (34)
+ : +- * ColumnarToRow (33)
+ : +- CometProject (32)
+ : +- CometFilter (31)
+ : +- CometScan parquet spark_catalog.default.store (30)
+ +- BroadcastExchange (40)
+ +- * ColumnarToRow (39)
+ +- CometFilter (38)
+ +- CometScan parquet spark_catalog.default.customer (37)
+
+
+(1) Scan parquet spark_catalog.default.store_returns
+Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)]
+PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1))
+
+(3) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#6, d_year#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [d_date_sk#6, d_year#7]
+Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6))
+
+(5) CometProject
+Input [2]: [d_date_sk#6, d_year#7]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(6) CometBroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: [d_date_sk#6]
+
+(7) CometBroadcastHashJoin
+Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4]
+Right output [1]: [d_date_sk#6]
+Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight
+
+(8) CometProject
+Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6]
+Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
+
+(9) CometHashAggregate
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3]
+Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
+Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))]
+
+(10) ColumnarToRow [codegen id : 1]
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8]
+
+(11) Exchange
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8]
+Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(12) HashAggregate [codegen id : 7]
+Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8]
+Keys [2]: [sr_customer_sk#1, sr_store_sk#2]
+Functions [1]: [sum(UnscaledValue(sr_return_amt#3))]
+Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9]
+Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12]
+
+(13) Filter [codegen id : 7]
+Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12]
+Condition : isnotnull(ctr_total_return#12)
+
+(14) Scan parquet spark_catalog.default.store_returns
+Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#17)]
+PushedFilters: [IsNotNull(sr_store_sk)]
+ReadSchema: struct
+
+(15) CometFilter
+Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16]
+Condition : isnotnull(sr_store_sk#14)
+
+(16) ReusedExchange [Reuses operator id: 6]
+Output [1]: [d_date_sk#18]
+
+(17) CometBroadcastHashJoin
+Left output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16]
+Right output [1]: [d_date_sk#18]
+Arguments: [sr_returned_date_sk#16], [d_date_sk#18], Inner, BuildRight
+
+(18) CometProject
+Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#18]
+Arguments: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15], [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15]
+
+(19) CometHashAggregate
+Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15]
+Keys [2]: [sr_customer_sk#13, sr_store_sk#14]
+Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))]
+
+(20) ColumnarToRow [codegen id : 2]
+Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19]
+
+(21) Exchange
+Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19]
+Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(22) HashAggregate [codegen id : 3]
+Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19]
+Keys [2]: [sr_customer_sk#13, sr_store_sk#14]
+Functions [1]: [sum(UnscaledValue(sr_return_amt#15))]
+Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9]
+Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21]
+
+(23) HashAggregate [codegen id : 3]
+Input [2]: [ctr_store_sk#20, ctr_total_return#21]
+Keys [1]: [ctr_store_sk#20]
+Functions [1]: [partial_avg(ctr_total_return#21)]
+Aggregate Attributes [2]: [sum#22, count#23]
+Results [3]: [ctr_store_sk#20, sum#24, count#25]
+
+(24) Exchange
+Input [3]: [ctr_store_sk#20, sum#24, count#25]
+Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(25) HashAggregate [codegen id : 4]
+Input [3]: [ctr_store_sk#20, sum#24, count#25]
+Keys [1]: [ctr_store_sk#20]
+Functions [1]: [avg(ctr_total_return#21)]
+Aggregate Attributes [1]: [avg(ctr_total_return#21)#26]
+Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]
+
+(26) Filter [codegen id : 4]
+Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]
+Condition : isnotnull((avg(ctr_total_return) * 1.2)#27)
+
+(27) BroadcastExchange
+Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4]
+
+(28) BroadcastHashJoin [codegen id : 7]
+Left keys [1]: [ctr_store_sk#11]
+Right keys [1]: [ctr_store_sk#20]
+Join type: Inner
+Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27)
+
+(29) Project [codegen id : 7]
+Output [2]: [ctr_customer_sk#10, ctr_store_sk#11]
+Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20]
+
+(30) Scan parquet spark_catalog.default.store
+Output [2]: [s_store_sk#28, s_state#29]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(31) CometFilter
+Input [2]: [s_store_sk#28, s_state#29]
+Condition : ((isnotnull(s_state#29) AND (s_state#29 = TN)) AND isnotnull(s_store_sk#28))
+
+(32) CometProject
+Input [2]: [s_store_sk#28, s_state#29]
+Arguments: [s_store_sk#28], [s_store_sk#28]
+
+(33) ColumnarToRow [codegen id : 5]
+Input [1]: [s_store_sk#28]
+
+(34) BroadcastExchange
+Input [1]: [s_store_sk#28]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5]
+
+(35) BroadcastHashJoin [codegen id : 7]
+Left keys [1]: [ctr_store_sk#11]
+Right keys [1]: [s_store_sk#28]
+Join type: Inner
+Join condition: None
+
+(36) Project [codegen id : 7]
+Output [1]: [ctr_customer_sk#10]
+Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28]
+
+(37) Scan parquet spark_catalog.default.customer
+Output [2]: [c_customer_sk#30, c_customer_id#31]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk)]
+ReadSchema: struct
+
+(38) CometFilter
+Input [2]: [c_customer_sk#30, c_customer_id#31]
+Condition : isnotnull(c_customer_sk#30)
+
+(39) ColumnarToRow [codegen id : 6]
+Input [2]: [c_customer_sk#30, c_customer_id#31]
+
+(40) BroadcastExchange
+Input [2]: [c_customer_sk#30, c_customer_id#31]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6]
+
+(41) BroadcastHashJoin [codegen id : 7]
+Left keys [1]: [ctr_customer_sk#10]
+Right keys [1]: [c_customer_sk#30]
+Join type: Inner
+Join condition: None
+
+(42) Project [codegen id : 7]
+Output [1]: [c_customer_id#31]
+Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#31]
+
+(43) TakeOrderedAndProject
+Input [1]: [c_customer_id#31]
+Arguments: 100, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5
+BroadcastExchange (48)
++- * ColumnarToRow (47)
+ +- CometProject (46)
+ +- CometFilter (45)
+ +- CometScan parquet spark_catalog.default.date_dim (44)
+
+
+(44) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#6, d_year#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(45) CometFilter
+Input [2]: [d_date_sk#6, d_year#7]
+Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6))
+
+(46) CometProject
+Input [2]: [d_date_sk#6, d_year#7]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(47) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#6]
+
+(48) BroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
+
+Subquery:2 Hosting operator id = 14 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt
new file mode 100644
index 000000000..dc0fb18f3
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt
@@ -0,0 +1,69 @@
+TakeOrderedAndProject [c_customer_id]
+ WholeStageCodegen (7)
+ Project [c_customer_id]
+ BroadcastHashJoin [ctr_customer_sk,c_customer_sk]
+ Project [ctr_customer_sk]
+ BroadcastHashJoin [ctr_store_sk,s_store_sk]
+ Project [ctr_customer_sk,ctr_store_sk]
+ BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)]
+ Filter [ctr_total_return]
+ HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_store_sk] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometProject [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk,sr_customer_sk]
+ CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #3
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (4)
+ Filter [(avg(ctr_total_return) * 1.2)]
+ HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count]
+ InputAdapter
+ Exchange [ctr_store_sk] #5
+ WholeStageCodegen (3)
+ HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count]
+ HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum]
+ InputAdapter
+ Exchange [sr_customer_sk,sr_store_sk] #6
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometProject [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk]
+ CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #3
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (5)
+ ColumnarToRow
+ InputAdapter
+ CometProject [s_store_sk]
+ CometFilter [s_state,s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
+ InputAdapter
+ BroadcastExchange #8
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [c_customer_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt
new file mode 100644
index 000000000..4a29b7260
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt
@@ -0,0 +1,297 @@
+== Physical Plan ==
+TakeOrderedAndProject (45)
++- * HashAggregate (44)
+ +- Exchange (43)
+ +- * HashAggregate (42)
+ +- * Project (41)
+ +- * BroadcastHashJoin Inner BuildRight (40)
+ :- * Project (35)
+ : +- * BroadcastHashJoin Inner BuildRight (34)
+ : :- * Project (28)
+ : : +- * Filter (27)
+ : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26)
+ : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19)
+ : : : :- * ColumnarToRow (12)
+ : : : : +- CometBroadcastHashJoin (11)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.customer (1)
+ : : : : +- CometBroadcastExchange (10)
+ : : : : +- CometProject (9)
+ : : : : +- CometBroadcastHashJoin (8)
+ : : : : :- CometScan parquet spark_catalog.default.store_sales (3)
+ : : : : +- CometBroadcastExchange (7)
+ : : : : +- CometProject (6)
+ : : : : +- CometFilter (5)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (4)
+ : : : +- BroadcastExchange (18)
+ : : : +- * ColumnarToRow (17)
+ : : : +- CometProject (16)
+ : : : +- CometBroadcastHashJoin (15)
+ : : : :- CometScan parquet spark_catalog.default.web_sales (13)
+ : : : +- ReusedExchange (14)
+ : : +- BroadcastExchange (25)
+ : : +- * ColumnarToRow (24)
+ : : +- CometProject (23)
+ : : +- CometBroadcastHashJoin (22)
+ : : :- CometScan parquet spark_catalog.default.catalog_sales (20)
+ : : +- ReusedExchange (21)
+ : +- BroadcastExchange (33)
+ : +- * ColumnarToRow (32)
+ : +- CometProject (31)
+ : +- CometFilter (30)
+ : +- CometScan parquet spark_catalog.default.customer_address (29)
+ +- BroadcastExchange (39)
+ +- * ColumnarToRow (38)
+ +- CometFilter (37)
+ +- CometScan parquet spark_catalog.default.customer_demographics (36)
+
+
+(1) Scan parquet spark_catalog.default.customer
+Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4))
+
+(3) Scan parquet spark_catalog.default.store_sales
+Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)]
+ReadSchema: struct
+
+(4) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#9, d_year#10, d_moy#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(5) CometFilter
+Input [3]: [d_date_sk#9, d_year#10, d_moy#11]
+Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9))
+
+(6) CometProject
+Input [3]: [d_date_sk#9, d_year#10, d_moy#11]
+Arguments: [d_date_sk#9], [d_date_sk#9]
+
+(7) CometBroadcastExchange
+Input [1]: [d_date_sk#9]
+Arguments: [d_date_sk#9]
+
+(8) CometBroadcastHashJoin
+Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Right output [1]: [d_date_sk#9]
+Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight
+
+(9) CometProject
+Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9]
+Arguments: [ss_customer_sk#6], [ss_customer_sk#6]
+
+(10) CometBroadcastExchange
+Input [1]: [ss_customer_sk#6]
+Arguments: [ss_customer_sk#6]
+
+(11) CometBroadcastHashJoin
+Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Right output [1]: [ss_customer_sk#6]
+Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight
+
+(12) ColumnarToRow [codegen id : 5]
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+
+(13) Scan parquet spark_catalog.default.web_sales
+Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)]
+ReadSchema: struct
+
+(14) ReusedExchange [Reuses operator id: 7]
+Output [1]: [d_date_sk#15]
+
+(15) CometBroadcastHashJoin
+Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13]
+Right output [1]: [d_date_sk#15]
+Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight
+
+(16) CometProject
+Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15]
+Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12]
+
+(17) ColumnarToRow [codegen id : 1]
+Input [1]: [ws_bill_customer_sk#12]
+
+(18) BroadcastExchange
+Input [1]: [ws_bill_customer_sk#12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1]
+
+(19) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [ws_bill_customer_sk#12]
+Join type: ExistenceJoin(exists#2)
+Join condition: None
+
+(20) Scan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)]
+ReadSchema: struct
+
+(21) ReusedExchange [Reuses operator id: 7]
+Output [1]: [d_date_sk#19]
+
+(22) CometBroadcastHashJoin
+Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#19]
+Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight
+
+(23) CometProject
+Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19]
+Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16]
+
+(24) ColumnarToRow [codegen id : 2]
+Input [1]: [cs_ship_customer_sk#16]
+
+(25) BroadcastExchange
+Input [1]: [cs_ship_customer_sk#16]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+(26) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [cs_ship_customer_sk#16]
+Join type: ExistenceJoin(exists#1)
+Join condition: None
+
+(27) Filter [codegen id : 5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+Condition : (exists#2 OR exists#1)
+
+(28) Project [codegen id : 5]
+Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+
+(29) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#20, ca_county#21]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(30) CometFilter
+Input [2]: [ca_address_sk#20, ca_county#21]
+Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20))
+
+(31) CometProject
+Input [2]: [ca_address_sk#20, ca_county#21]
+Arguments: [ca_address_sk#20], [ca_address_sk#20]
+
+(32) ColumnarToRow [codegen id : 3]
+Input [1]: [ca_address_sk#20]
+
+(33) BroadcastExchange
+Input [1]: [ca_address_sk#20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+(34) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_addr_sk#5]
+Right keys [1]: [ca_address_sk#20]
+Join type: Inner
+Join condition: None
+
+(35) Project [codegen id : 5]
+Output [1]: [c_current_cdemo_sk#4]
+Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20]
+
+(36) Scan parquet spark_catalog.default.customer_demographics
+Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_demo_sk)]
+ReadSchema: struct
+
+(37) CometFilter
+Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+Condition : isnotnull(cd_demo_sk#22)
+
+(38) ColumnarToRow [codegen id : 4]
+Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+
+(39) BroadcastExchange
+Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
+
+(40) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_cdemo_sk#4]
+Right keys [1]: [cd_demo_sk#22]
+Join type: Inner
+Join condition: None
+
+(41) Project [codegen id : 5]
+Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+
+(42) HashAggregate [codegen id : 5]
+Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+Functions [1]: [partial_count(1)]
+Aggregate Attributes [1]: [count#31]
+Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32]
+
+(43) Exchange
+Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32]
+Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+
+(44) HashAggregate [codegen id : 6]
+Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32]
+Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30]
+Functions [1]: [count(1)]
+Aggregate Attributes [1]: [count(1)#33]
+Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#33 AS cnt1#34, cd_purchase_estimate#26, count(1)#33 AS cnt2#35, cd_credit_rating#27, count(1)#33 AS cnt3#36, cd_dep_count#28, count(1)#33 AS cnt4#37, cd_dep_employed_count#29, count(1)#33 AS cnt5#38, cd_dep_college_count#30, count(1)#33 AS cnt6#39]
+
+(45) TakeOrderedAndProject
+Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39]
+Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8
+BroadcastExchange (50)
++- * ColumnarToRow (49)
+ +- CometProject (48)
+ +- CometFilter (47)
+ +- CometScan parquet spark_catalog.default.date_dim (46)
+
+
+(46) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#9, d_year#10, d_moy#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(47) CometFilter
+Input [3]: [d_date_sk#9, d_year#10, d_moy#11]
+Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9))
+
+(48) CometProject
+Input [3]: [d_date_sk#9, d_year#10, d_moy#11]
+Arguments: [d_date_sk#9], [d_date_sk#9]
+
+(49) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#9]
+
+(50) BroadcastExchange
+Input [1]: [d_date_sk#9]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
+
+Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8
+
+Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt
new file mode 100644
index 000000000..19243e359
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt
@@ -0,0 +1,71 @@
+TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6]
+ WholeStageCodegen (6)
+ HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count]
+ InputAdapter
+ Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1
+ WholeStageCodegen (5)
+ HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count]
+ Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ Project [c_current_cdemo_sk]
+ BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ Project [c_current_cdemo_sk,c_current_addr_sk]
+ Filter [exists,exists]
+ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk]
+ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
+ ColumnarToRow
+ InputAdapter
+ CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk]
+ CometBroadcastExchange #2
+ CometProject [ss_customer_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #4
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #4
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometProject [cs_ship_customer_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #4
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ca_address_sk]
+ CometFilter [ca_county,ca_address_sk]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
+ InputAdapter
+ BroadcastExchange #8
+ WholeStageCodegen (4)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [cd_demo_sk]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt
new file mode 100644
index 000000000..1c5d5222e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt
@@ -0,0 +1,478 @@
+== Physical Plan ==
+TakeOrderedAndProject (72)
++- * Project (71)
+ +- * BroadcastHashJoin Inner BuildRight (70)
+ :- * Project (54)
+ : +- * BroadcastHashJoin Inner BuildRight (53)
+ : :- * Project (36)
+ : : +- * BroadcastHashJoin Inner BuildRight (35)
+ : : :- * Filter (17)
+ : : : +- * HashAggregate (16)
+ : : : +- Exchange (15)
+ : : : +- * ColumnarToRow (14)
+ : : : +- CometHashAggregate (13)
+ : : : +- CometProject (12)
+ : : : +- CometBroadcastHashJoin (11)
+ : : : :- CometProject (7)
+ : : : : +- CometBroadcastHashJoin (6)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.customer (1)
+ : : : : +- CometBroadcastExchange (5)
+ : : : : +- CometFilter (4)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (3)
+ : : : +- CometBroadcastExchange (10)
+ : : : +- CometFilter (9)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (8)
+ : : +- BroadcastExchange (34)
+ : : +- * HashAggregate (33)
+ : : +- Exchange (32)
+ : : +- * ColumnarToRow (31)
+ : : +- CometHashAggregate (30)
+ : : +- CometProject (29)
+ : : +- CometBroadcastHashJoin (28)
+ : : :- CometProject (24)
+ : : : +- CometBroadcastHashJoin (23)
+ : : : :- CometFilter (19)
+ : : : : +- CometScan parquet spark_catalog.default.customer (18)
+ : : : +- CometBroadcastExchange (22)
+ : : : +- CometFilter (21)
+ : : : +- CometScan parquet spark_catalog.default.store_sales (20)
+ : : +- CometBroadcastExchange (27)
+ : : +- CometFilter (26)
+ : : +- CometScan parquet spark_catalog.default.date_dim (25)
+ : +- BroadcastExchange (52)
+ : +- * Filter (51)
+ : +- * HashAggregate (50)
+ : +- Exchange (49)
+ : +- * ColumnarToRow (48)
+ : +- CometHashAggregate (47)
+ : +- CometProject (46)
+ : +- CometBroadcastHashJoin (45)
+ : :- CometProject (43)
+ : : +- CometBroadcastHashJoin (42)
+ : : :- CometFilter (38)
+ : : : +- CometScan parquet spark_catalog.default.customer (37)
+ : : +- CometBroadcastExchange (41)
+ : : +- CometFilter (40)
+ : : +- CometScan parquet spark_catalog.default.web_sales (39)
+ : +- ReusedExchange (44)
+ +- BroadcastExchange (69)
+ +- * HashAggregate (68)
+ +- Exchange (67)
+ +- * ColumnarToRow (66)
+ +- CometHashAggregate (65)
+ +- CometProject (64)
+ +- CometBroadcastHashJoin (63)
+ :- CometProject (61)
+ : +- CometBroadcastHashJoin (60)
+ : :- CometFilter (56)
+ : : +- CometScan parquet spark_catalog.default.customer (55)
+ : +- CometBroadcastExchange (59)
+ : +- CometFilter (58)
+ : +- CometScan parquet spark_catalog.default.web_sales (57)
+ +- ReusedExchange (62)
+
+
+(1) Scan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2))
+
+(3) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)]
+PushedFilters: [IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Condition : isnotnull(ss_customer_sk#9)
+
+(5) CometBroadcastExchange
+Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+
+(6) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Right output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight
+
+(7) CometProject
+Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+
+(8) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#14, d_year#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [2]: [d_date_sk#14, d_year#15]
+Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14))
+
+(10) CometBroadcastExchange
+Input [2]: [d_date_sk#14, d_year#15]
+Arguments: [d_date_sk#14, d_year#15]
+
+(11) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12]
+Right output [2]: [d_date_sk#14, d_year#15]
+Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight
+
+(12) CometProject
+Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15]
+Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15]
+
+(13) CometHashAggregate
+Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15]
+Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+
+(14) ColumnarToRow [codegen id : 1]
+Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16]
+
+(15) Exchange
+Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16]
+Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(16) HashAggregate [codegen id : 8]
+Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16]
+Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#17]
+Results [2]: [c_customer_id#2 AS customer_id#18, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#17,18,2) AS year_total#19]
+
+(17) Filter [codegen id : 8]
+Input [2]: [customer_id#18, year_total#19]
+Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00))
+
+(18) Scan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(19) CometFilter
+Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27]
+Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_customer_id#21))
+
+(20) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)]
+PushedFilters: [IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(21) CometFilter
+Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+Condition : isnotnull(ss_customer_sk#28)
+
+(22) CometBroadcastExchange
+Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+Arguments: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+
+(23) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27]
+Right output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+Arguments: [c_customer_sk#20], [ss_customer_sk#28], Inner, BuildRight
+
+(24) CometProject
+Input [12]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31], [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+
+(25) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#33, d_year#34]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [d_date_sk#33, d_year#34]
+Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33))
+
+(27) CometBroadcastExchange
+Input [2]: [d_date_sk#33, d_year#34]
+Arguments: [d_date_sk#33, d_year#34]
+
+(28) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31]
+Right output [2]: [d_date_sk#33, d_year#34]
+Arguments: [ss_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight
+
+(29) CometProject
+Input [12]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31, d_date_sk#33, d_year#34]
+Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34], [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34]
+
+(30) CometHashAggregate
+Input [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34]
+Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))]
+
+(31) ColumnarToRow [codegen id : 2]
+Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35]
+
+(32) Exchange
+Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35]
+Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(33) HashAggregate [codegen id : 3]
+Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35]
+Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))#17]
+Results [3]: [c_customer_id#21 AS customer_id#36, c_preferred_cust_flag#24 AS customer_preferred_cust_flag#37, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))#17,18,2) AS year_total#38]
+
+(34) BroadcastExchange
+Input [3]: [customer_id#36, customer_preferred_cust_flag#37, year_total#38]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3]
+
+(35) BroadcastHashJoin [codegen id : 8]
+Left keys [1]: [customer_id#18]
+Right keys [1]: [customer_id#36]
+Join type: Inner
+Join condition: None
+
+(36) Project [codegen id : 8]
+Output [4]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38]
+Input [5]: [customer_id#18, year_total#19, customer_id#36, customer_preferred_cust_flag#37, year_total#38]
+
+(37) Scan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(38) CometFilter
+Input [8]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46]
+Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_customer_id#40))
+
+(39) Scan parquet spark_catalog.default.web_sales
+Output [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)]
+PushedFilters: [IsNotNull(ws_bill_customer_sk)]
+ReadSchema: struct
+
+(40) CometFilter
+Input [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+Condition : isnotnull(ws_bill_customer_sk#47)
+
+(41) CometBroadcastExchange
+Input [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+Arguments: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+
+(42) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46]
+Right output [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+Arguments: [c_customer_sk#39], [ws_bill_customer_sk#47], Inner, BuildRight
+
+(43) CometProject
+Input [12]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50], [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+
+(44) ReusedExchange [Reuses operator id: 10]
+Output [2]: [d_date_sk#52, d_year#53]
+
+(45) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50]
+Right output [2]: [d_date_sk#52, d_year#53]
+Arguments: [ws_sold_date_sk#50], [d_date_sk#52], Inner, BuildRight
+
+(46) CometProject
+Input [12]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50, d_date_sk#52, d_year#53]
+Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53], [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53]
+
+(47) CometHashAggregate
+Input [10]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53]
+Keys [8]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))]
+
+(48) ColumnarToRow [codegen id : 4]
+Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54]
+
+(49) Exchange
+Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54]
+Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+
+(50) HashAggregate [codegen id : 5]
+Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54]
+Keys [8]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))#55]
+Results [2]: [c_customer_id#40 AS customer_id#56, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))#55,18,2) AS year_total#57]
+
+(51) Filter [codegen id : 5]
+Input [2]: [customer_id#56, year_total#57]
+Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00))
+
+(52) BroadcastExchange
+Input [2]: [customer_id#56, year_total#57]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5]
+
+(53) BroadcastHashJoin [codegen id : 8]
+Left keys [1]: [customer_id#18]
+Right keys [1]: [customer_id#56]
+Join type: Inner
+Join condition: None
+
+(54) Project [codegen id : 8]
+Output [5]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, year_total#57]
+Input [6]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, customer_id#56, year_total#57]
+
+(55) Scan parquet spark_catalog.default.customer
+Output [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)]
+ReadSchema: struct
+
+(56) CometFilter
+Input [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65]
+Condition : (isnotnull(c_customer_sk#58) AND isnotnull(c_customer_id#59))
+
+(57) Scan parquet spark_catalog.default.web_sales
+Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)]
+PushedFilters: [IsNotNull(ws_bill_customer_sk)]
+ReadSchema: struct
+
+(58) CometFilter
+Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+Condition : isnotnull(ws_bill_customer_sk#66)
+
+(59) CometBroadcastExchange
+Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+Arguments: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+
+(60) CometBroadcastHashJoin
+Left output [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65]
+Right output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+Arguments: [c_customer_sk#58], [ws_bill_customer_sk#66], Inner, BuildRight
+
+(61) CometProject
+Input [12]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+
+(62) ReusedExchange [Reuses operator id: 27]
+Output [2]: [d_date_sk#71, d_year#72]
+
+(63) CometBroadcastHashJoin
+Left output [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69]
+Right output [2]: [d_date_sk#71, d_year#72]
+Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight
+
+(64) CometProject
+Input [12]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69, d_date_sk#71, d_year#72]
+Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72]
+
+(65) CometHashAggregate
+Input [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72]
+Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))]
+
+(66) ColumnarToRow [codegen id : 6]
+Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73]
+
+(67) Exchange
+Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73]
+Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+
+(68) HashAggregate [codegen id : 7]
+Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73]
+Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))#55]
+Results [2]: [c_customer_id#59 AS customer_id#74, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))#55,18,2) AS year_total#75]
+
+(69) BroadcastExchange
+Input [2]: [customer_id#74, year_total#75]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7]
+
+(70) BroadcastHashJoin [codegen id : 8]
+Left keys [1]: [customer_id#18]
+Right keys [1]: [customer_id#74]
+Join type: Inner
+Join condition: (CASE WHEN (year_total#57 > 0.00) THEN (year_total#75 / year_total#57) END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#38 / year_total#19) END)
+
+(71) Project [codegen id : 8]
+Output [1]: [customer_preferred_cust_flag#37]
+Input [7]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, year_total#57, customer_id#74, year_total#75]
+
+(72) TakeOrderedAndProject
+Input [1]: [customer_preferred_cust_flag#37]
+Arguments: 100, [customer_preferred_cust_flag#37 ASC NULLS FIRST], [customer_preferred_cust_flag#37]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13
+BroadcastExchange (76)
++- * ColumnarToRow (75)
+ +- CometFilter (74)
+ +- CometScan parquet spark_catalog.default.date_dim (73)
+
+
+(73) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#14, d_year#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(74) CometFilter
+Input [2]: [d_date_sk#14, d_year#15]
+Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14))
+
+(75) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#14, d_year#15]
+
+(76) BroadcastExchange
+Input [2]: [d_date_sk#14, d_year#15]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8]
+
+Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32
+BroadcastExchange (80)
++- * ColumnarToRow (79)
+ +- CometFilter (78)
+ +- CometScan parquet spark_catalog.default.date_dim (77)
+
+
+(77) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#33, d_year#34]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(78) CometFilter
+Input [2]: [d_date_sk#33, d_year#34]
+Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33))
+
+(79) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#33, d_year#34]
+
+(80) BroadcastExchange
+Input [2]: [d_date_sk#33, d_year#34]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9]
+
+Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#13
+
+Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#32
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt
new file mode 100644
index 000000000..dd1a52206
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt
@@ -0,0 +1,107 @@
+TakeOrderedAndProject [customer_preferred_cust_flag]
+ WholeStageCodegen (8)
+ Project [customer_preferred_cust_flag]
+ BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total]
+ Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total]
+ BroadcastHashJoin [customer_id,customer_id]
+ Project [customer_id,year_total,customer_preferred_cust_flag,year_total]
+ BroadcastHashJoin [customer_id,customer_id]
+ Filter [year_total]
+ HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum]
+ InputAdapter
+ Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometFilter [c_customer_sk,c_customer_id]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometBroadcastExchange #2
+ CometFilter [ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #4
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (3)
+ HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum]
+ InputAdapter
+ Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometFilter [c_customer_sk,c_customer_id]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometBroadcastExchange #7
+ CometFilter [ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #8
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #9
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ InputAdapter
+ BroadcastExchange #10
+ WholeStageCodegen (5)
+ Filter [year_total]
+ HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
+ InputAdapter
+ Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11
+ WholeStageCodegen (4)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
+ CometFilter [c_customer_sk,c_customer_id]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometBroadcastExchange #12
+ CometFilter [ws_bill_customer_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk,d_year] #4
+ InputAdapter
+ BroadcastExchange #13
+ WholeStageCodegen (7)
+ HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
+ InputAdapter
+ Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
+ CometFilter [c_customer_sk,c_customer_id]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
+ CometBroadcastExchange #15
+ CometFilter [ws_bill_customer_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ ReusedExchange [d_date_sk,d_year] #9
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt
new file mode 100644
index 000000000..160f0c593
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt
@@ -0,0 +1,161 @@
+== Physical Plan ==
+TakeOrderedAndProject (22)
++- * Project (21)
+ +- Window (20)
+ +- * Sort (19)
+ +- Exchange (18)
+ +- * HashAggregate (17)
+ +- Exchange (16)
+ +- * ColumnarToRow (15)
+ +- CometHashAggregate (14)
+ +- CometProject (13)
+ +- CometBroadcastHashJoin (12)
+ :- CometProject (7)
+ : +- CometBroadcastHashJoin (6)
+ : :- CometFilter (2)
+ : : +- CometScan parquet spark_catalog.default.web_sales (1)
+ : +- CometBroadcastExchange (5)
+ : +- CometFilter (4)
+ : +- CometScan parquet spark_catalog.default.item (3)
+ +- CometBroadcastExchange (11)
+ +- CometProject (10)
+ +- CometFilter (9)
+ +- CometScan parquet spark_catalog.default.date_dim (8)
+
+
+(1) Scan parquet spark_catalog.default.web_sales
+Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Condition : isnotnull(ws_item_sk#1)
+
+(3) Scan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5))
+
+(5) CometBroadcastExchange
+Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+
+(6) CometBroadcastHashJoin
+Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3]
+Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight
+
+(7) CometProject
+Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+
+(8) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#11, d_date#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [2]: [d_date_sk#11, d_date#12]
+Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11))
+
+(10) CometProject
+Input [2]: [d_date_sk#11, d_date#12]
+Arguments: [d_date_sk#11], [d_date_sk#11]
+
+(11) CometBroadcastExchange
+Input [1]: [d_date_sk#11]
+Arguments: [d_date_sk#11]
+
+(12) CometBroadcastHashJoin
+Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Right output [1]: [d_date_sk#11]
+Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight
+
+(13) CometProject
+Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11]
+Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+
+(14) CometHashAggregate
+Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8]
+Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))]
+
+(15) ColumnarToRow [codegen id : 1]
+Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13]
+
+(16) Exchange
+Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13]
+Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(17) HashAggregate [codegen id : 2]
+Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13]
+Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8]
+Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14]
+Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6]
+
+(18) Exchange
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6]
+Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(19) Sort [codegen id : 3]
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6]
+Arguments: [i_class#9 ASC NULLS FIRST], false, 0
+
+(20) Window
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6]
+Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9]
+
+(21) Project [codegen id : 4]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6]
+Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17]
+
+(22) TakeOrderedAndProject
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6]
+Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4
+BroadcastExchange (27)
++- * ColumnarToRow (26)
+ +- CometProject (25)
+ +- CometFilter (24)
+ +- CometScan parquet spark_catalog.default.date_dim (23)
+
+
+(23) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#11, d_date#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(24) CometFilter
+Input [2]: [d_date_sk#11, d_date#12]
+Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11))
+
+(25) CometProject
+Input [2]: [d_date_sk#11, d_date#12]
+Arguments: [d_date_sk#11], [d_date_sk#11]
+
+(26) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#11]
+
+(27) BroadcastExchange
+Input [1]: [d_date_sk#11]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt
new file mode 100644
index 000000000..905a35c8b
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt
@@ -0,0 +1,38 @@
+TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue]
+ WholeStageCodegen (4)
+ Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id]
+ InputAdapter
+ Window [_w0,i_class]
+ WholeStageCodegen (3)
+ Sort [i_class]
+ InputAdapter
+ Exchange [i_class] #1
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum]
+ InputAdapter
+ Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price]
+ CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometFilter [ws_item_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
+ CometBroadcastExchange #4
+ CometFilter [i_category,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastExchange #5
+ CometProject [d_date_sk]
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt
new file mode 100644
index 000000000..1cda12261
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt
@@ -0,0 +1,228 @@
+== Physical Plan ==
+* HashAggregate (33)
++- Exchange (32)
+ +- * ColumnarToRow (31)
+ +- CometHashAggregate (30)
+ +- CometProject (29)
+ +- CometBroadcastHashJoin (28)
+ :- CometProject (24)
+ : +- CometBroadcastHashJoin (23)
+ : :- CometProject (19)
+ : : +- CometBroadcastHashJoin (18)
+ : : :- CometProject (13)
+ : : : +- CometBroadcastHashJoin (12)
+ : : : :- CometProject (7)
+ : : : : +- CometBroadcastHashJoin (6)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : +- CometBroadcastExchange (5)
+ : : : : +- CometFilter (4)
+ : : : : +- CometScan parquet spark_catalog.default.store (3)
+ : : : +- CometBroadcastExchange (11)
+ : : : +- CometProject (10)
+ : : : +- CometFilter (9)
+ : : : +- CometScan parquet spark_catalog.default.customer_address (8)
+ : : +- CometBroadcastExchange (17)
+ : : +- CometProject (16)
+ : : +- CometFilter (15)
+ : : +- CometScan parquet spark_catalog.default.date_dim (14)
+ : +- CometBroadcastExchange (22)
+ : +- CometFilter (21)
+ : +- CometScan parquet spark_catalog.default.customer_demographics (20)
+ +- CometBroadcastExchange (27)
+ +- CometFilter (26)
+ +- CometScan parquet spark_catalog.default.household_demographics (25)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)]
+PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))]
+ReadSchema: struct
+
+(2) CometFilter
+Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00))))
+
+(3) Scan parquet spark_catalog.default.store
+Output [1]: [s_store_sk#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [1]: [s_store_sk#12]
+Condition : isnotnull(s_store_sk#12)
+
+(5) CometBroadcastExchange
+Input [1]: [s_store_sk#12]
+Arguments: [s_store_sk#12]
+
+(6) CometBroadcastHashJoin
+Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Right output [1]: [s_store_sk#12]
+Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight
+
+(7) CometProject
+Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+
+(8) Scan parquet spark_catalog.default.customer_address
+Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))]
+ReadSchema: struct
+
+(9) CometFilter
+Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15]
+Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (TX,OH) OR ca_state#14 IN (OR,NM,KY)) OR ca_state#14 IN (VA,TX,MS)))
+
+(10) CometProject
+Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15]
+Arguments: [ca_address_sk#13, ca_state#14], [ca_address_sk#13, ca_state#14]
+
+(11) CometBroadcastExchange
+Input [2]: [ca_address_sk#13, ca_state#14]
+Arguments: [ca_address_sk#13, ca_state#14]
+
+(12) CometBroadcastHashJoin
+Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10]
+Right output [2]: [ca_address_sk#13, ca_state#14]
+Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight
+
+(13) CometProject
+Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#14]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10]
+
+(14) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#16, d_year#17]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(15) CometFilter
+Input [2]: [d_date_sk#16, d_year#17]
+Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16))
+
+(16) CometProject
+Input [2]: [d_date_sk#16, d_year#17]
+Arguments: [d_date_sk#16], [d_date_sk#16]
+
+(17) CometBroadcastExchange
+Input [1]: [d_date_sk#16]
+Arguments: [d_date_sk#16]
+
+(18) CometBroadcastHashJoin
+Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10]
+Right output [1]: [d_date_sk#16]
+Arguments: [ss_sold_date_sk#10], [d_date_sk#16], Inner, BuildRight
+
+(19) CometProject
+Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#16]
+Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+
+(20) Scan parquet spark_catalog.default.customer_demographics
+Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))]
+ReadSchema: struct
+
+(21) CometFilter
+Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20]
+Condition : (isnotnull(cd_demo_sk#18) AND ((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) OR ((cd_marital_status#19 = S) AND (cd_education_status#20 = College ))) OR ((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree ))))
+
+(22) CometBroadcastExchange
+Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20]
+Arguments: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20]
+
+(23) CometBroadcastHashJoin
+Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+Right output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20]
+Arguments: [ss_cdemo_sk#1], [cd_demo_sk#18], Inner, ((((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#19 = S) AND (cd_education_status#20 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight
+
+(24) CometProject
+Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#19, cd_education_status#20]
+Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20]
+
+(25) Scan parquet spark_catalog.default.household_demographics
+Output [2]: [hd_demo_sk#21, hd_dep_count#22]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/household_demographics]
+PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [hd_demo_sk#21, hd_dep_count#22]
+Condition : (isnotnull(hd_demo_sk#21) AND ((hd_dep_count#22 = 3) OR (hd_dep_count#22 = 1)))
+
+(27) CometBroadcastExchange
+Input [2]: [hd_demo_sk#21, hd_dep_count#22]
+Arguments: [hd_demo_sk#21, hd_dep_count#22]
+
+(28) CometBroadcastHashJoin
+Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20]
+Right output [2]: [hd_demo_sk#21, hd_dep_count#22]
+Arguments: [ss_hdemo_sk#2], [hd_demo_sk#21], Inner, (((((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#22 = 3)) OR (((((cd_marital_status#19 = S) AND (cd_education_status#20 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#22 = 1))) OR (((((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#22 = 1))), BuildRight
+
+(29) CometProject
+Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20, hd_demo_sk#21, hd_dep_count#22]
+Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+
+(30) CometHashAggregate
+Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8]
+Keys: []
+Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))]
+
+(31) ColumnarToRow [codegen id : 1]
+Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29]
+
+(32) Exchange
+Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1]
+
+(33) HashAggregate [codegen id : 2]
+Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29]
+Keys: []
+Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))]
+Aggregate Attributes [4]: [avg(ss_quantity#5)#30, avg(UnscaledValue(ss_ext_sales_price#7))#31, avg(UnscaledValue(ss_ext_wholesale_cost#8))#32, sum(UnscaledValue(ss_ext_wholesale_cost#8))#33]
+Results [4]: [avg(ss_quantity#5)#30 AS avg(ss_quantity)#34, cast((avg(UnscaledValue(ss_ext_sales_price#7))#31 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#35, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#36, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#33,17,2) AS sum(ss_ext_wholesale_cost)#37]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11
+BroadcastExchange (38)
++- * ColumnarToRow (37)
+ +- CometProject (36)
+ +- CometFilter (35)
+ +- CometScan parquet spark_catalog.default.date_dim (34)
+
+
+(34) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#16, d_year#17]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(35) CometFilter
+Input [2]: [d_date_sk#16, d_year#17]
+Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16))
+
+(36) CometProject
+Input [2]: [d_date_sk#16, d_year#17]
+Arguments: [d_date_sk#16], [d_date_sk#16]
+
+(37) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#16]
+
+(38) BroadcastExchange
+Input [1]: [d_date_sk#16]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt
new file mode 100644
index 000000000..4de403664
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt
@@ -0,0 +1,45 @@
+WholeStageCodegen (2)
+ HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum]
+ InputAdapter
+ Exchange #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count]
+ CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status]
+ CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit]
+ CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price]
+ CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #3
+ CometFilter [s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk]
+ CometBroadcastExchange #4
+ CometProject [ca_address_sk,ca_state]
+ CometFilter [ca_country,ca_address_sk,ca_state]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
+ CometBroadcastExchange #5
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #6
+ CometFilter [cd_demo_sk,cd_marital_status,cd_education_status]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status]
+ CometBroadcastExchange #7
+ CometFilter [hd_demo_sk,hd_dep_count]
+ CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt
new file mode 100644
index 000000000..01b77a0d3
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt
@@ -0,0 +1,788 @@
+== Physical Plan ==
+TakeOrderedAndProject (105)
++- * HashAggregate (104)
+ +- Exchange (103)
+ +- * HashAggregate (102)
+ +- * Expand (101)
+ +- Union (100)
+ :- * Project (67)
+ : +- * Filter (66)
+ : +- * HashAggregate (65)
+ : +- Exchange (64)
+ : +- * HashAggregate (63)
+ : +- * Project (62)
+ : +- * BroadcastHashJoin Inner BuildRight (61)
+ : :- * Project (59)
+ : : +- * BroadcastHashJoin Inner BuildRight (58)
+ : : :- * BroadcastHashJoin LeftSemi BuildRight (51)
+ : : : :- * ColumnarToRow (3)
+ : : : : +- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : +- BroadcastExchange (50)
+ : : : +- * Project (49)
+ : : : +- * BroadcastHashJoin Inner BuildRight (48)
+ : : : :- * ColumnarToRow (6)
+ : : : : +- CometFilter (5)
+ : : : : +- CometScan parquet spark_catalog.default.item (4)
+ : : : +- BroadcastExchange (47)
+ : : : +- * BroadcastHashJoin LeftSemi BuildRight (46)
+ : : : :- * HashAggregate (35)
+ : : : : +- Exchange (34)
+ : : : : +- * ColumnarToRow (33)
+ : : : : +- CometHashAggregate (32)
+ : : : : +- CometProject (31)
+ : : : : +- CometBroadcastHashJoin (30)
+ : : : : :- CometProject (28)
+ : : : : : +- CometBroadcastHashJoin (27)
+ : : : : : :- CometFilter (8)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7)
+ : : : : : +- CometBroadcastExchange (26)
+ : : : : : +- CometBroadcastHashJoin (25)
+ : : : : : :- CometFilter (10)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (9)
+ : : : : : +- CometBroadcastExchange (24)
+ : : : : : +- CometProject (23)
+ : : : : : +- CometBroadcastHashJoin (22)
+ : : : : : :- CometProject (17)
+ : : : : : : +- CometBroadcastHashJoin (16)
+ : : : : : : :- CometFilter (12)
+ : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11)
+ : : : : : : +- CometBroadcastExchange (15)
+ : : : : : : +- CometFilter (14)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (13)
+ : : : : : +- CometBroadcastExchange (21)
+ : : : : : +- CometProject (20)
+ : : : : : +- CometFilter (19)
+ : : : : : +- CometScan parquet spark_catalog.default.date_dim (18)
+ : : : : +- ReusedExchange (29)
+ : : : +- BroadcastExchange (45)
+ : : : +- * ColumnarToRow (44)
+ : : : +- CometProject (43)
+ : : : +- CometBroadcastHashJoin (42)
+ : : : :- CometProject (40)
+ : : : : +- CometBroadcastHashJoin (39)
+ : : : : :- CometFilter (37)
+ : : : : : +- CometScan parquet spark_catalog.default.web_sales (36)
+ : : : : +- ReusedExchange (38)
+ : : : +- ReusedExchange (41)
+ : : +- BroadcastExchange (57)
+ : : +- * BroadcastHashJoin LeftSemi BuildRight (56)
+ : : :- * ColumnarToRow (54)
+ : : : +- CometFilter (53)
+ : : : +- CometScan parquet spark_catalog.default.item (52)
+ : : +- ReusedExchange (55)
+ : +- ReusedExchange (60)
+ :- * Project (83)
+ : +- * Filter (82)
+ : +- * HashAggregate (81)
+ : +- Exchange (80)
+ : +- * HashAggregate (79)
+ : +- * Project (78)
+ : +- * BroadcastHashJoin Inner BuildRight (77)
+ : :- * Project (75)
+ : : +- * BroadcastHashJoin Inner BuildRight (74)
+ : : :- * BroadcastHashJoin LeftSemi BuildRight (72)
+ : : : :- * ColumnarToRow (70)
+ : : : : +- CometFilter (69)
+ : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68)
+ : : : +- ReusedExchange (71)
+ : : +- ReusedExchange (73)
+ : +- ReusedExchange (76)
+ +- * Project (99)
+ +- * Filter (98)
+ +- * HashAggregate (97)
+ +- Exchange (96)
+ +- * HashAggregate (95)
+ +- * Project (94)
+ +- * BroadcastHashJoin Inner BuildRight (93)
+ :- * Project (91)
+ : +- * BroadcastHashJoin Inner BuildRight (90)
+ : :- * BroadcastHashJoin LeftSemi BuildRight (88)
+ : : :- * ColumnarToRow (86)
+ : : : +- CometFilter (85)
+ : : : +- CometScan parquet spark_catalog.default.web_sales (84)
+ : : +- ReusedExchange (87)
+ : +- ReusedExchange (89)
+ +- ReusedExchange (92)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Condition : isnotnull(ss_item_sk#1)
+
+(3) ColumnarToRow [codegen id : 11]
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+
+(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
+
+(5) CometFilter
+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))
+
+(6) ColumnarToRow [codegen id : 4]
+Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
+
+(7) Scan parquet spark_catalog.default.store_sales
+Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(8) CometFilter
+Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
+Condition : isnotnull(ss_item_sk#10)
+
+(9) Scan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+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
+
+(10) CometFilter
+Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16))
+
+(11) Scan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_item_sk#17, cs_sold_date_sk#18]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(12) CometFilter
+Input [2]: [cs_item_sk#17, cs_sold_date_sk#18]
+Condition : isnotnull(cs_item_sk#17)
+
+(13) Scan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(14) CometFilter
+Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Condition : isnotnull(i_item_sk#20)
+
+(15) CometBroadcastExchange
+Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(16) CometBroadcastHashJoin
+Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18]
+Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight
+
+(17) CometProject
+Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(18) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#24, d_year#25]
+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
+
+(19) CometFilter
+Input [2]: [d_date_sk#24, d_year#25]
+Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24))
+
+(20) CometProject
+Input [2]: [d_date_sk#24, d_year#25]
+Arguments: [d_date_sk#24], [d_date_sk#24]
+
+(21) CometBroadcastExchange
+Input [1]: [d_date_sk#24]
+Arguments: [d_date_sk#24]
+
+(22) CometBroadcastHashJoin
+Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23]
+Right output [1]: [d_date_sk#24]
+Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight
+
+(23) CometProject
+Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24]
+Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(24) CometBroadcastExchange
+Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(25) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight
+
+(26) CometBroadcastExchange
+Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+
+(27) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
+Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight
+
+(28) CometProject
+Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16]
+
+(29) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#26]
+
+(30) CometBroadcastHashJoin
+Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16]
+Right output [1]: [d_date_sk#26]
+Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight
+
+(31) CometProject
+Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26]
+Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29]
+
+(32) CometHashAggregate
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Keys [3]: [brand_id#27, class_id#28, category_id#29]
+Functions: []
+
+(33) ColumnarToRow [codegen id : 1]
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+
+(34) Exchange
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(35) HashAggregate [codegen id : 3]
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Keys [3]: [brand_id#27, class_id#28, category_id#29]
+Functions: []
+Aggregate Attributes: []
+Results [3]: [brand_id#27, class_id#28, category_id#29]
+
+(36) Scan parquet spark_catalog.default.web_sales
+Output [2]: [ws_item_sk#30, ws_sold_date_sk#31]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(37) CometFilter
+Input [2]: [ws_item_sk#30, ws_sold_date_sk#31]
+Condition : isnotnull(ws_item_sk#30)
+
+(38) ReusedExchange [Reuses operator id: 15]
+Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(39) CometBroadcastHashJoin
+Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31]
+Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36]
+Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight
+
+(40) CometProject
+Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36]
+Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(41) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#37]
+
+(42) CometBroadcastHashJoin
+Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36]
+Right output [1]: [d_date_sk#37]
+Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight
+
+(43) CometProject
+Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37]
+Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(44) ColumnarToRow [codegen id : 2]
+Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(45) BroadcastExchange
+Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36]
+Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2]
+
+(46) BroadcastHashJoin [codegen id : 3]
+Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)]
+Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)]
+Join type: LeftSemi
+Join condition: None
+
+(47) BroadcastExchange
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3]
+
+(48) BroadcastHashJoin [codegen id : 4]
+Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
+Right keys [3]: [brand_id#27, class_id#28, category_id#29]
+Join type: Inner
+Join condition: None
+
+(49) Project [codegen id : 4]
+Output [1]: [i_item_sk#6 AS ss_item_sk#38]
+Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29]
+
+(50) BroadcastExchange
+Input [1]: [ss_item_sk#38]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
+
+(51) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [ss_item_sk#1]
+Right keys [1]: [ss_item_sk#38]
+Join type: LeftSemi
+Join condition: None
+
+(52) Scan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(53) CometFilter
+Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+Condition : isnotnull(i_item_sk#39)
+
+(54) ColumnarToRow [codegen id : 9]
+Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+
+(55) ReusedExchange [Reuses operator id: 50]
+Output [1]: [ss_item_sk#38]
+
+(56) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [i_item_sk#39]
+Right keys [1]: [ss_item_sk#38]
+Join type: LeftSemi
+Join condition: None
+
+(57) BroadcastExchange
+Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+
+(58) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [ss_item_sk#1]
+Right keys [1]: [i_item_sk#39]
+Join type: Inner
+Join condition: None
+
+(59) Project [codegen id : 11]
+Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42]
+Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+
+(60) ReusedExchange [Reuses operator id: 127]
+Output [1]: [d_date_sk#43]
+
+(61) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [ss_sold_date_sk#4]
+Right keys [1]: [d_date_sk#43]
+Join type: Inner
+Join condition: None
+
+(62) Project [codegen id : 11]
+Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42]
+Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43]
+
+(63) HashAggregate [codegen id : 11]
+Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42]
+Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
+Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46]
+Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49]
+
+(64) Exchange
+Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49]
+Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+
+(65) HashAggregate [codegen id : 12]
+Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49]
+Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42]
+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))#50, count(1)#51]
+Results [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#52, count(1)#51 AS number_sales#53]
+
+(66) Filter [codegen id : 12]
+Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53]
+Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6))))
+
+(67) Project [codegen id : 12]
+Output [6]: [sales#52, number_sales#53, store AS channel#56, i_brand_id#40 AS i_brand_id#57, i_class_id#41 AS i_class_id#58, i_category_id#42 AS i_category_id#59]
+Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53]
+
+(68) Scan parquet spark_catalog.default.catalog_sales
+Output [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(69) CometFilter
+Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63]
+Condition : isnotnull(cs_item_sk#60)
+
+(70) ColumnarToRow [codegen id : 23]
+Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63]
+
+(71) ReusedExchange [Reuses operator id: 50]
+Output [1]: [ss_item_sk#65]
+
+(72) BroadcastHashJoin [codegen id : 23]
+Left keys [1]: [cs_item_sk#60]
+Right keys [1]: [ss_item_sk#65]
+Join type: LeftSemi
+Join condition: None
+
+(73) ReusedExchange [Reuses operator id: 57]
+Output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69]
+
+(74) BroadcastHashJoin [codegen id : 23]
+Left keys [1]: [cs_item_sk#60]
+Right keys [1]: [i_item_sk#66]
+Join type: Inner
+Join condition: None
+
+(75) Project [codegen id : 23]
+Output [6]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69]
+Input [8]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69]
+
+(76) ReusedExchange [Reuses operator id: 127]
+Output [1]: [d_date_sk#70]
+
+(77) BroadcastHashJoin [codegen id : 23]
+Left keys [1]: [cs_sold_date_sk#63]
+Right keys [1]: [d_date_sk#70]
+Join type: Inner
+Join condition: None
+
+(78) Project [codegen id : 23]
+Output [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69]
+Input [7]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69, d_date_sk#70]
+
+(79) HashAggregate [codegen id : 23]
+Input [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69]
+Keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69]
+Functions [2]: [partial_sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), partial_count(1)]
+Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73]
+Results [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#74, isEmpty#75, count#76]
+
+(80) Exchange
+Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#74, isEmpty#75, count#76]
+Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+
+(81) HashAggregate [codegen id : 24]
+Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#74, isEmpty#75, count#76]
+Keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69]
+Functions [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), count(1)]
+Aggregate Attributes [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#77, count(1)#78]
+Results [5]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#77 AS sales#79, count(1)#78 AS number_sales#80]
+
+(82) Filter [codegen id : 24]
+Input [5]: [i_brand_id#67, i_class_id#68, i_category_id#69, sales#79, number_sales#80]
+Condition : (isnotnull(sales#79) AND (cast(sales#79 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6))))
+
+(83) Project [codegen id : 24]
+Output [6]: [sales#79, number_sales#80, catalog AS channel#81, i_brand_id#67, i_class_id#68, i_category_id#69]
+Input [5]: [i_brand_id#67, i_class_id#68, i_category_id#69, sales#79, number_sales#80]
+
+(84) Scan parquet spark_catalog.default.web_sales
+Output [4]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#86)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(85) CometFilter
+Input [4]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85]
+Condition : isnotnull(ws_item_sk#82)
+
+(86) ColumnarToRow [codegen id : 35]
+Input [4]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85]
+
+(87) ReusedExchange [Reuses operator id: 50]
+Output [1]: [ss_item_sk#87]
+
+(88) BroadcastHashJoin [codegen id : 35]
+Left keys [1]: [ws_item_sk#82]
+Right keys [1]: [ss_item_sk#87]
+Join type: LeftSemi
+Join condition: None
+
+(89) ReusedExchange [Reuses operator id: 57]
+Output [4]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91]
+
+(90) BroadcastHashJoin [codegen id : 35]
+Left keys [1]: [ws_item_sk#82]
+Right keys [1]: [i_item_sk#88]
+Join type: Inner
+Join condition: None
+
+(91) Project [codegen id : 35]
+Output [6]: [ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85, i_brand_id#89, i_class_id#90, i_category_id#91]
+Input [8]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91]
+
+(92) ReusedExchange [Reuses operator id: 127]
+Output [1]: [d_date_sk#92]
+
+(93) BroadcastHashJoin [codegen id : 35]
+Left keys [1]: [ws_sold_date_sk#85]
+Right keys [1]: [d_date_sk#92]
+Join type: Inner
+Join condition: None
+
+(94) Project [codegen id : 35]
+Output [5]: [ws_quantity#83, ws_list_price#84, i_brand_id#89, i_class_id#90, i_category_id#91]
+Input [7]: [ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85, i_brand_id#89, i_class_id#90, i_category_id#91, d_date_sk#92]
+
+(95) HashAggregate [codegen id : 35]
+Input [5]: [ws_quantity#83, ws_list_price#84, i_brand_id#89, i_class_id#90, i_category_id#91]
+Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91]
+Functions [2]: [partial_sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84)), partial_count(1)]
+Aggregate Attributes [3]: [sum#93, isEmpty#94, count#95]
+Results [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98]
+
+(96) Exchange
+Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98]
+Arguments: hashpartitioning(i_brand_id#89, i_class_id#90, i_category_id#91, 5), ENSURE_REQUIREMENTS, [plan_id=8]
+
+(97) HashAggregate [codegen id : 36]
+Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98]
+Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91]
+Functions [2]: [sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84))#99, count(1)#100]
+Results [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84))#99 AS sales#101, count(1)#100 AS number_sales#102]
+
+(98) Filter [codegen id : 36]
+Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#101, number_sales#102]
+Condition : (isnotnull(sales#101) AND (cast(sales#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6))))
+
+(99) Project [codegen id : 36]
+Output [6]: [sales#101, number_sales#102, web AS channel#103, i_brand_id#89, i_class_id#90, i_category_id#91]
+Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#101, number_sales#102]
+
+(100) Union
+
+(101) Expand [codegen id : 37]
+Input [6]: [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59]
+Arguments: [[sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59, 0], [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, null, 1], [sales#52, number_sales#53, channel#56, i_brand_id#57, null, null, 3], [sales#52, number_sales#53, channel#56, null, null, null, 7], [sales#52, number_sales#53, null, null, null, null, 15]], [sales#52, number_sales#53, channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108]
+
+(102) HashAggregate [codegen id : 37]
+Input [7]: [sales#52, number_sales#53, channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108]
+Keys [5]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108]
+Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)]
+Aggregate Attributes [3]: [sum#109, isEmpty#110, sum#111]
+Results [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114]
+
+(103) Exchange
+Input [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114]
+Arguments: hashpartitioning(channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, 5), ENSURE_REQUIREMENTS, [plan_id=9]
+
+(104) HashAggregate [codegen id : 38]
+Input [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114]
+Keys [5]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108]
+Functions [2]: [sum(sales#52), sum(number_sales#53)]
+Aggregate Attributes [2]: [sum(sales#52)#115, sum(number_sales#53)#116]
+Results [6]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales#52)#115 AS sum(sales)#117, sum(number_sales#53)#116 AS sum(number_sales)#118]
+
+(105) TakeOrderedAndProject
+Input [6]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales)#117, sum(number_sales)#118]
+Arguments: 100, [channel#104 ASC NULLS FIRST, i_brand_id#105 ASC NULLS FIRST, i_class_id#106 ASC NULLS FIRST, i_category_id#107 ASC NULLS FIRST], [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales)#117, sum(number_sales)#118]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55]
+* HashAggregate (122)
++- Exchange (121)
+ +- * ColumnarToRow (120)
+ +- CometHashAggregate (119)
+ +- CometUnion (118)
+ :- CometProject (109)
+ : +- CometBroadcastHashJoin (108)
+ : :- CometScan parquet spark_catalog.default.store_sales (106)
+ : +- ReusedExchange (107)
+ :- CometProject (113)
+ : +- CometBroadcastHashJoin (112)
+ : :- CometScan parquet spark_catalog.default.catalog_sales (110)
+ : +- ReusedExchange (111)
+ +- CometProject (117)
+ +- CometBroadcastHashJoin (116)
+ :- CometScan parquet spark_catalog.default.web_sales (114)
+ +- ReusedExchange (115)
+
+
+(106) Scan parquet spark_catalog.default.store_sales
+Output [3]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#121), dynamicpruningexpression(ss_sold_date_sk#121 IN dynamicpruning#122)]
+ReadSchema: struct
+
+(107) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#123]
+
+(108) CometBroadcastHashJoin
+Left output [3]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121]
+Right output [1]: [d_date_sk#123]
+Arguments: [ss_sold_date_sk#121], [d_date_sk#123], Inner, BuildRight
+
+(109) CometProject
+Input [4]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121, d_date_sk#123]
+Arguments: [quantity#124, list_price#125], [ss_quantity#119 AS quantity#124, ss_list_price#120 AS list_price#125]
+
+(110) Scan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#128), dynamicpruningexpression(cs_sold_date_sk#128 IN dynamicpruning#129)]
+ReadSchema: struct
+
+(111) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#130]
+
+(112) CometBroadcastHashJoin
+Left output [3]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128]
+Right output [1]: [d_date_sk#130]
+Arguments: [cs_sold_date_sk#128], [d_date_sk#130], Inner, BuildRight
+
+(113) CometProject
+Input [4]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128, d_date_sk#130]
+Arguments: [quantity#131, list_price#132], [cs_quantity#126 AS quantity#131, cs_list_price#127 AS list_price#132]
+
+(114) Scan parquet spark_catalog.default.web_sales
+Output [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#136)]
+ReadSchema: struct
+
+(115) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#137]
+
+(116) CometBroadcastHashJoin
+Left output [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135]
+Right output [1]: [d_date_sk#137]
+Arguments: [ws_sold_date_sk#135], [d_date_sk#137], Inner, BuildRight
+
+(117) CometProject
+Input [4]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135, d_date_sk#137]
+Arguments: [quantity#138, list_price#139], [ws_quantity#133 AS quantity#138, ws_list_price#134 AS list_price#139]
+
+(118) CometUnion
+Child 0 Input [2]: [quantity#124, list_price#125]
+Child 1 Input [2]: [quantity#131, list_price#132]
+Child 2 Input [2]: [quantity#138, list_price#139]
+
+(119) CometHashAggregate
+Input [2]: [quantity#124, list_price#125]
+Keys: []
+Functions [1]: [partial_avg((cast(quantity#124 as decimal(10,0)) * list_price#125))]
+
+(120) ColumnarToRow [codegen id : 1]
+Input [2]: [sum#140, count#141]
+
+(121) Exchange
+Input [2]: [sum#140, count#141]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10]
+
+(122) HashAggregate [codegen id : 2]
+Input [2]: [sum#140, count#141]
+Keys: []
+Functions [1]: [avg((cast(quantity#124 as decimal(10,0)) * list_price#125))]
+Aggregate Attributes [1]: [avg((cast(quantity#124 as decimal(10,0)) * list_price#125))#142]
+Results [1]: [avg((cast(quantity#124 as decimal(10,0)) * list_price#125))#142 AS average_sales#143]
+
+Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#121 IN dynamicpruning#12
+
+Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#128 IN dynamicpruning#12
+
+Subquery:4 Hosting operator id = 114 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#12
+
+Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
+BroadcastExchange (127)
++- * ColumnarToRow (126)
+ +- CometProject (125)
+ +- CometFilter (124)
+ +- CometScan parquet spark_catalog.default.date_dim (123)
+
+
+(123) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#43, d_year#144, d_moy#145]
+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
+
+(124) CometFilter
+Input [3]: [d_date_sk#43, d_year#144, d_moy#145]
+Condition : ((((isnotnull(d_year#144) AND isnotnull(d_moy#145)) AND (d_year#144 = 2001)) AND (d_moy#145 = 11)) AND isnotnull(d_date_sk#43))
+
+(125) CometProject
+Input [3]: [d_date_sk#43, d_year#144, d_moy#145]
+Arguments: [d_date_sk#43], [d_date_sk#43]
+
+(126) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#43]
+
+(127) BroadcastExchange
+Input [1]: [d_date_sk#43]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11]
+
+Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (132)
++- * ColumnarToRow (131)
+ +- CometProject (130)
+ +- CometFilter (129)
+ +- CometScan parquet spark_catalog.default.date_dim (128)
+
+
+(128) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#26, d_year#146]
+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
+
+(129) CometFilter
+Input [2]: [d_date_sk#26, d_year#146]
+Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 <= 2001)) AND isnotnull(d_date_sk#26))
+
+(130) CometProject
+Input [2]: [d_date_sk#26, d_year#146]
+Arguments: [d_date_sk#26], [d_date_sk#26]
+
+(131) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#26]
+
+(132) BroadcastExchange
+Input [1]: [d_date_sk#26]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12]
+
+Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12
+
+Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12
+
+Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55]
+
+Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#5
+
+Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55]
+
+Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt
new file mode 100644
index 000000000..e3dfa631b
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt
@@ -0,0 +1,190 @@
+TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)]
+ WholeStageCodegen (38)
+ 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 (37)
+ 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 (12)
+ Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
+ Filter [sales]
+ Subquery #3
+ WholeStageCodegen (2)
+ HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
+ InputAdapter
+ Exchange #14
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [quantity,list_price]
+ CometUnion
+ CometProject [ss_quantity,ss_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ ReusedExchange [d_date_sk] #11
+ CometProject [cs_quantity,cs_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ ReusedExchange [d_date_sk] #11
+ CometProject [ws_quantity,ws_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ ReusedExchange [d_date_sk] #11
+ 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 (11)
+ 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_sold_date_sk,d_date_sk]
+ Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ BroadcastHashJoin [ss_item_sk,ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ss_item_sk]
+ CometScan 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)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (4)
+ Project [i_item_sk]
+ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (3)
+ BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+ HashAggregate [brand_id,class_id,category_id]
+ InputAdapter
+ Exchange [brand_id,class_id,category_id] #6
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometFilter [ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #7
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #8
+ CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange #9
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometFilter [cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ CometBroadcastExchange #10
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange #11
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ ReusedExchange [d_date_sk] #11
+ InputAdapter
+ BroadcastExchange #12
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometFilter [ws_item_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10
+ ReusedExchange [d_date_sk] #11
+ InputAdapter
+ BroadcastExchange #13
+ WholeStageCodegen (9)
+ BroadcastHashJoin [i_item_sk,ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ InputAdapter
+ ReusedExchange [ss_item_sk] #4
+ InputAdapter
+ ReusedExchange [d_date_sk] #3
+ WholeStageCodegen (24)
+ 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] #15
+ WholeStageCodegen (23)
+ 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_sold_date_sk,d_date_sk]
+ Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ BroadcastHashJoin [cs_item_sk,i_item_sk]
+ BroadcastHashJoin [cs_item_sk,ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ InputAdapter
+ ReusedExchange [ss_item_sk] #4
+ InputAdapter
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
+ InputAdapter
+ ReusedExchange [d_date_sk] #3
+ WholeStageCodegen (36)
+ 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] #16
+ WholeStageCodegen (35)
+ 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_sold_date_sk,d_date_sk]
+ Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ BroadcastHashJoin [ws_item_sk,i_item_sk]
+ BroadcastHashJoin [ws_item_sk,ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ws_item_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ InputAdapter
+ ReusedExchange [ss_item_sk] #4
+ InputAdapter
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13
+ InputAdapter
+ ReusedExchange [d_date_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt
new file mode 100644
index 000000000..4270241a8
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt
@@ -0,0 +1,743 @@
+== Physical Plan ==
+TakeOrderedAndProject (84)
++- * BroadcastHashJoin Inner BuildRight (83)
+ :- * Filter (66)
+ : +- * HashAggregate (65)
+ : +- Exchange (64)
+ : +- * HashAggregate (63)
+ : +- * Project (62)
+ : +- * BroadcastHashJoin Inner BuildRight (61)
+ : :- * Project (59)
+ : : +- * BroadcastHashJoin Inner BuildRight (58)
+ : : :- * BroadcastHashJoin LeftSemi BuildRight (51)
+ : : : :- * ColumnarToRow (3)
+ : : : : +- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : +- BroadcastExchange (50)
+ : : : +- * Project (49)
+ : : : +- * BroadcastHashJoin Inner BuildRight (48)
+ : : : :- * ColumnarToRow (6)
+ : : : : +- CometFilter (5)
+ : : : : +- CometScan parquet spark_catalog.default.item (4)
+ : : : +- BroadcastExchange (47)
+ : : : +- * BroadcastHashJoin LeftSemi BuildRight (46)
+ : : : :- * HashAggregate (35)
+ : : : : +- Exchange (34)
+ : : : : +- * ColumnarToRow (33)
+ : : : : +- CometHashAggregate (32)
+ : : : : +- CometProject (31)
+ : : : : +- CometBroadcastHashJoin (30)
+ : : : : :- CometProject (28)
+ : : : : : +- CometBroadcastHashJoin (27)
+ : : : : : :- CometFilter (8)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7)
+ : : : : : +- CometBroadcastExchange (26)
+ : : : : : +- CometBroadcastHashJoin (25)
+ : : : : : :- CometFilter (10)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (9)
+ : : : : : +- CometBroadcastExchange (24)
+ : : : : : +- CometProject (23)
+ : : : : : +- CometBroadcastHashJoin (22)
+ : : : : : :- CometProject (17)
+ : : : : : : +- CometBroadcastHashJoin (16)
+ : : : : : : :- CometFilter (12)
+ : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11)
+ : : : : : : +- CometBroadcastExchange (15)
+ : : : : : : +- CometFilter (14)
+ : : : : : : +- CometScan parquet spark_catalog.default.item (13)
+ : : : : : +- CometBroadcastExchange (21)
+ : : : : : +- CometProject (20)
+ : : : : : +- CometFilter (19)
+ : : : : : +- CometScan parquet spark_catalog.default.date_dim (18)
+ : : : : +- ReusedExchange (29)
+ : : : +- BroadcastExchange (45)
+ : : : +- * ColumnarToRow (44)
+ : : : +- CometProject (43)
+ : : : +- CometBroadcastHashJoin (42)
+ : : : :- CometProject (40)
+ : : : : +- CometBroadcastHashJoin (39)
+ : : : : :- CometFilter (37)
+ : : : : : +- CometScan parquet spark_catalog.default.web_sales (36)
+ : : : : +- ReusedExchange (38)
+ : : : +- ReusedExchange (41)
+ : : +- BroadcastExchange (57)
+ : : +- * BroadcastHashJoin LeftSemi BuildRight (56)
+ : : :- * ColumnarToRow (54)
+ : : : +- CometFilter (53)
+ : : : +- CometScan parquet spark_catalog.default.item (52)
+ : : +- ReusedExchange (55)
+ : +- ReusedExchange (60)
+ +- BroadcastExchange (82)
+ +- * Filter (81)
+ +- * HashAggregate (80)
+ +- Exchange (79)
+ +- * HashAggregate (78)
+ +- * Project (77)
+ +- * BroadcastHashJoin Inner BuildRight (76)
+ :- * Project (74)
+ : +- * BroadcastHashJoin Inner BuildRight (73)
+ : :- * BroadcastHashJoin LeftSemi BuildRight (71)
+ : : :- * ColumnarToRow (69)
+ : : : +- CometFilter (68)
+ : : : +- CometScan parquet spark_catalog.default.store_sales (67)
+ : : +- ReusedExchange (70)
+ : +- ReusedExchange (72)
+ +- ReusedExchange (75)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+Condition : isnotnull(ss_item_sk#1)
+
+(3) ColumnarToRow [codegen id : 11]
+Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4]
+
+(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
+
+(5) CometFilter
+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))
+
+(6) ColumnarToRow [codegen id : 4]
+Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9]
+
+(7) Scan parquet spark_catalog.default.store_sales
+Output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(8) CometFilter
+Input [2]: [ss_item_sk#10, ss_sold_date_sk#11]
+Condition : isnotnull(ss_item_sk#10)
+
+(9) Scan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+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
+
+(10) CometFilter
+Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16))
+
+(11) Scan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_item_sk#17, cs_sold_date_sk#18]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(12) CometFilter
+Input [2]: [cs_item_sk#17, cs_sold_date_sk#18]
+Condition : isnotnull(cs_item_sk#17)
+
+(13) Scan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(14) CometFilter
+Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Condition : isnotnull(i_item_sk#20)
+
+(15) CometBroadcastExchange
+Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(16) CometBroadcastHashJoin
+Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18]
+Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight
+
+(17) CometProject
+Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(18) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#24, d_year#25]
+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
+
+(19) CometFilter
+Input [2]: [d_date_sk#24, d_year#25]
+Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24))
+
+(20) CometProject
+Input [2]: [d_date_sk#24, d_year#25]
+Arguments: [d_date_sk#24], [d_date_sk#24]
+
+(21) CometBroadcastExchange
+Input [1]: [d_date_sk#24]
+Arguments: [d_date_sk#24]
+
+(22) CometBroadcastHashJoin
+Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23]
+Right output [1]: [d_date_sk#24]
+Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight
+
+(23) CometProject
+Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24]
+Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(24) CometBroadcastExchange
+Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23]
+
+(25) CometBroadcastHashJoin
+Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23]
+Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight
+
+(26) CometBroadcastExchange
+Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+
+(27) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11]
+Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight
+
+(28) CometProject
+Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16]
+Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16]
+
+(29) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#26]
+
+(30) CometBroadcastHashJoin
+Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16]
+Right output [1]: [d_date_sk#26]
+Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight
+
+(31) CometProject
+Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26]
+Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29]
+
+(32) CometHashAggregate
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Keys [3]: [brand_id#27, class_id#28, category_id#29]
+Functions: []
+
+(33) ColumnarToRow [codegen id : 1]
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+
+(34) Exchange
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(35) HashAggregate [codegen id : 3]
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Keys [3]: [brand_id#27, class_id#28, category_id#29]
+Functions: []
+Aggregate Attributes: []
+Results [3]: [brand_id#27, class_id#28, category_id#29]
+
+(36) Scan parquet spark_catalog.default.web_sales
+Output [2]: [ws_item_sk#30, ws_sold_date_sk#31]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)]
+PushedFilters: [IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(37) CometFilter
+Input [2]: [ws_item_sk#30, ws_sold_date_sk#31]
+Condition : isnotnull(ws_item_sk#30)
+
+(38) ReusedExchange [Reuses operator id: 15]
+Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(39) CometBroadcastHashJoin
+Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31]
+Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36]
+Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight
+
+(40) CometProject
+Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36]
+Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(41) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#37]
+
+(42) CometBroadcastHashJoin
+Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36]
+Right output [1]: [d_date_sk#37]
+Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight
+
+(43) CometProject
+Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37]
+Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(44) ColumnarToRow [codegen id : 2]
+Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36]
+
+(45) BroadcastExchange
+Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36]
+Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2]
+
+(46) BroadcastHashJoin [codegen id : 3]
+Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)]
+Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)]
+Join type: LeftSemi
+Join condition: None
+
+(47) BroadcastExchange
+Input [3]: [brand_id#27, class_id#28, category_id#29]
+Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3]
+
+(48) BroadcastHashJoin [codegen id : 4]
+Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9]
+Right keys [3]: [brand_id#27, class_id#28, category_id#29]
+Join type: Inner
+Join condition: None
+
+(49) Project [codegen id : 4]
+Output [1]: [i_item_sk#6 AS ss_item_sk#38]
+Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29]
+
+(50) BroadcastExchange
+Input [1]: [ss_item_sk#38]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
+
+(51) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [ss_item_sk#1]
+Right keys [1]: [ss_item_sk#38]
+Join type: LeftSemi
+Join condition: None
+
+(52) Scan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+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
+
+(53) CometFilter
+Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42))
+
+(54) ColumnarToRow [codegen id : 9]
+Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+
+(55) ReusedExchange [Reuses operator id: 50]
+Output [1]: [ss_item_sk#38]
+
+(56) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [i_item_sk#39]
+Right keys [1]: [ss_item_sk#38]
+Join type: LeftSemi
+Join condition: None
+
+(57) BroadcastExchange
+Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+
+(58) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [ss_item_sk#1]
+Right keys [1]: [i_item_sk#39]
+Join type: Inner
+Join condition: None
+
+(59) Project [codegen id : 11]
+Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42]
+Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42]
+
+(60) ReusedExchange [Reuses operator id: 106]
+Output [1]: [d_date_sk#43]
+
+(61) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [ss_sold_date_sk#4]
+Right keys [1]: [d_date_sk#43]
+Join type: Inner
+Join condition: None
+
+(62) Project [codegen id : 11]
+Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42]
+Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43]
+
+(63) HashAggregate [codegen id : 11]
+Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42]
+Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
+Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46]
+Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49]
+
+(64) Exchange
+Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49]
+Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+
+(65) HashAggregate [codegen id : 24]
+Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49]
+Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42]
+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))#50, count(1)#51]
+Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54]
+
+(66) Filter [codegen id : 24]
+Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54]
+Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6))))
+
+(67) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(68) CometFilter
+Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60]
+Condition : isnotnull(ss_item_sk#57)
+
+(69) ColumnarToRow [codegen id : 22]
+Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60]
+
+(70) ReusedExchange [Reuses operator id: 50]
+Output [1]: [ss_item_sk#62]
+
+(71) BroadcastHashJoin [codegen id : 22]
+Left keys [1]: [ss_item_sk#57]
+Right keys [1]: [ss_item_sk#62]
+Join type: LeftSemi
+Join condition: None
+
+(72) ReusedExchange [Reuses operator id: 57]
+Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66]
+
+(73) BroadcastHashJoin [codegen id : 22]
+Left keys [1]: [ss_item_sk#57]
+Right keys [1]: [i_item_sk#63]
+Join type: Inner
+Join condition: None
+
+(74) Project [codegen id : 22]
+Output [6]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66]
+Input [8]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66]
+
+(75) ReusedExchange [Reuses operator id: 120]
+Output [1]: [d_date_sk#67]
+
+(76) BroadcastHashJoin [codegen id : 22]
+Left keys [1]: [ss_sold_date_sk#60]
+Right keys [1]: [d_date_sk#67]
+Join type: Inner
+Join condition: None
+
+(77) Project [codegen id : 22]
+Output [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66]
+Input [7]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67]
+
+(78) HashAggregate [codegen id : 22]
+Input [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66]
+Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66]
+Functions [2]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), partial_count(1)]
+Aggregate Attributes [3]: [sum#68, isEmpty#69, count#70]
+Results [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73]
+
+(79) Exchange
+Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73]
+Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+
+(80) HashAggregate [codegen id : 23]
+Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73]
+Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66]
+Functions [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#74, count(1)#75]
+Results [6]: [store AS channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#74 AS sales#77, count(1)#75 AS number_sales#78]
+
+(81) Filter [codegen id : 23]
+Input [6]: [channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78]
+Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6))))
+
+(82) BroadcastExchange
+Input [6]: [channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78]
+Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=8]
+
+(83) BroadcastHashJoin [codegen id : 24]
+Left keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42]
+Right keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66]
+Join type: Inner
+Join condition: None
+
+(84) TakeOrderedAndProject
+Input [12]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78]
+Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56]
+* HashAggregate (101)
++- Exchange (100)
+ +- * ColumnarToRow (99)
+ +- CometHashAggregate (98)
+ +- CometUnion (97)
+ :- CometProject (88)
+ : +- CometBroadcastHashJoin (87)
+ : :- CometScan parquet spark_catalog.default.store_sales (85)
+ : +- ReusedExchange (86)
+ :- CometProject (92)
+ : +- CometBroadcastHashJoin (91)
+ : :- CometScan parquet spark_catalog.default.catalog_sales (89)
+ : +- ReusedExchange (90)
+ +- CometProject (96)
+ +- CometBroadcastHashJoin (95)
+ :- CometScan parquet spark_catalog.default.web_sales (93)
+ +- ReusedExchange (94)
+
+
+(85) Scan parquet spark_catalog.default.store_sales
+Output [3]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#81), dynamicpruningexpression(ss_sold_date_sk#81 IN dynamicpruning#82)]
+ReadSchema: struct
+
+(86) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#83]
+
+(87) CometBroadcastHashJoin
+Left output [3]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81]
+Right output [1]: [d_date_sk#83]
+Arguments: [ss_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight
+
+(88) CometProject
+Input [4]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81, d_date_sk#83]
+Arguments: [quantity#84, list_price#85], [ss_quantity#79 AS quantity#84, ss_list_price#80 AS list_price#85]
+
+(89) Scan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#88), dynamicpruningexpression(cs_sold_date_sk#88 IN dynamicpruning#89)]
+ReadSchema: struct
+
+(90) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#90]
+
+(91) CometBroadcastHashJoin
+Left output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88]
+Right output [1]: [d_date_sk#90]
+Arguments: [cs_sold_date_sk#88], [d_date_sk#90], Inner, BuildRight
+
+(92) CometProject
+Input [4]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88, d_date_sk#90]
+Arguments: [quantity#91, list_price#92], [cs_quantity#86 AS quantity#91, cs_list_price#87 AS list_price#92]
+
+(93) Scan parquet spark_catalog.default.web_sales
+Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)]
+ReadSchema: struct
+
+(94) ReusedExchange [Reuses operator id: 21]
+Output [1]: [d_date_sk#97]
+
+(95) CometBroadcastHashJoin
+Left output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95]
+Right output [1]: [d_date_sk#97]
+Arguments: [ws_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight
+
+(96) CometProject
+Input [4]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95, d_date_sk#97]
+Arguments: [quantity#98, list_price#99], [ws_quantity#93 AS quantity#98, ws_list_price#94 AS list_price#99]
+
+(97) CometUnion
+Child 0 Input [2]: [quantity#84, list_price#85]
+Child 1 Input [2]: [quantity#91, list_price#92]
+Child 2 Input [2]: [quantity#98, list_price#99]
+
+(98) CometHashAggregate
+Input [2]: [quantity#84, list_price#85]
+Keys: []
+Functions [1]: [partial_avg((cast(quantity#84 as decimal(10,0)) * list_price#85))]
+
+(99) ColumnarToRow [codegen id : 1]
+Input [2]: [sum#100, count#101]
+
+(100) Exchange
+Input [2]: [sum#100, count#101]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9]
+
+(101) HashAggregate [codegen id : 2]
+Input [2]: [sum#100, count#101]
+Keys: []
+Functions [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))]
+Aggregate Attributes [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#102]
+Results [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#102 AS average_sales#103]
+
+Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#81 IN dynamicpruning#12
+
+Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#88 IN dynamicpruning#12
+
+Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#12
+
+Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
+BroadcastExchange (106)
++- * ColumnarToRow (105)
+ +- CometProject (104)
+ +- CometFilter (103)
+ +- CometScan parquet spark_catalog.default.date_dim (102)
+
+
+(102) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#43, d_week_seq#104]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(103) CometFilter
+Input [2]: [d_date_sk#43, d_week_seq#104]
+Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#43))
+
+(104) CometProject
+Input [2]: [d_date_sk#43, d_week_seq#104]
+Arguments: [d_date_sk#43], [d_date_sk#43]
+
+(105) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#43]
+
+(106) BroadcastExchange
+Input [1]: [d_date_sk#43]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10]
+
+Subquery:6 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#105, [id=#106]
+* ColumnarToRow (110)
++- CometProject (109)
+ +- CometFilter (108)
+ +- CometScan parquet spark_catalog.default.date_dim (107)
+
+
+(107) Scan parquet spark_catalog.default.date_dim
+Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110]
+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
+
+(108) CometFilter
+Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110]
+Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 2000)) AND (d_moy#109 = 12)) AND (d_dom#110 = 11))
+
+(109) CometProject
+Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110]
+Arguments: [d_week_seq#107], [d_week_seq#107]
+
+(110) ColumnarToRow [codegen id : 1]
+Input [1]: [d_week_seq#107]
+
+Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12
+BroadcastExchange (115)
++- * ColumnarToRow (114)
+ +- CometProject (113)
+ +- CometFilter (112)
+ +- CometScan parquet spark_catalog.default.date_dim (111)
+
+
+(111) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#26, d_year#111]
+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
+
+(112) CometFilter
+Input [2]: [d_date_sk#26, d_year#111]
+Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1999)) AND (d_year#111 <= 2001)) AND isnotnull(d_date_sk#26))
+
+(113) CometProject
+Input [2]: [d_date_sk#26, d_year#111]
+Arguments: [d_date_sk#26], [d_date_sk#26]
+
+(114) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#26]
+
+(115) BroadcastExchange
+Input [1]: [d_date_sk#26]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11]
+
+Subquery:8 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12
+
+Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12
+
+Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56]
+
+Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61
+BroadcastExchange (120)
++- * ColumnarToRow (119)
+ +- CometProject (118)
+ +- CometFilter (117)
+ +- CometScan parquet spark_catalog.default.date_dim (116)
+
+
+(116) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#67, d_week_seq#112]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(117) CometFilter
+Input [2]: [d_date_sk#67, d_week_seq#112]
+Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#67))
+
+(118) CometProject
+Input [2]: [d_date_sk#67, d_week_seq#112]
+Arguments: [d_date_sk#67], [d_date_sk#67]
+
+(119) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#67]
+
+(120) BroadcastExchange
+Input [1]: [d_date_sk#67]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12]
+
+Subquery:12 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#113, [id=#114]
+* ColumnarToRow (124)
++- CometProject (123)
+ +- CometFilter (122)
+ +- CometScan parquet spark_catalog.default.date_dim (121)
+
+
+(121) Scan parquet spark_catalog.default.date_dim
+Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118]
+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
+
+(122) CometFilter
+Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118]
+Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1999)) AND (d_moy#117 = 12)) AND (d_dom#118 = 11))
+
+(123) CometProject
+Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118]
+Arguments: [d_week_seq#115], [d_week_seq#115]
+
+(124) ColumnarToRow [codegen id : 1]
+Input [1]: [d_week_seq#115]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt
new file mode 100644
index 000000000..799f74a36
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt
@@ -0,0 +1,178 @@
+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 (24)
+ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ Filter [sales]
+ Subquery #4
+ WholeStageCodegen (2)
+ HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
+ InputAdapter
+ Exchange #13
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [quantity,list_price]
+ CometUnion
+ CometProject [ss_quantity,ss_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk]
+ ReusedSubquery [d_date_sk] #3
+ ReusedExchange [d_date_sk] #10
+ CometProject [cs_quantity,cs_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #3
+ ReusedExchange [d_date_sk] #10
+ CometProject [ws_quantity,ws_list_price] [quantity,list_price]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #3
+ ReusedExchange [d_date_sk] #10
+ 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 (11)
+ 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_sold_date_sk,d_date_sk]
+ Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ BroadcastHashJoin [ss_item_sk,ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ss_item_sk]
+ CometScan 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)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_week_seq,d_date_sk]
+ Subquery #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_week_seq]
+ CometFilter [d_year,d_moy,d_dom]
+ CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+ InputAdapter
+ BroadcastExchange #3
+ WholeStageCodegen (4)
+ Project [i_item_sk]
+ BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (3)
+ BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id]
+ HashAggregate [brand_id,class_id,category_id]
+ InputAdapter
+ Exchange [brand_id,class_id,category_id] #5
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [brand_id,class_id,category_id]
+ CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometFilter [ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #3
+ BroadcastExchange #6
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #7
+ CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange #8
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometFilter [cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #3
+ CometBroadcastExchange #9
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange #10
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ ReusedExchange [d_date_sk] #10
+ InputAdapter
+ BroadcastExchange #11
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometProject [i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometFilter [ws_item_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #3
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ ReusedExchange [d_date_sk] #10
+ InputAdapter
+ BroadcastExchange #12
+ WholeStageCodegen (9)
+ BroadcastHashJoin [i_item_sk,ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id]
+ InputAdapter
+ ReusedExchange [ss_item_sk] #3
+ InputAdapter
+ ReusedExchange [d_date_sk] #2
+ InputAdapter
+ BroadcastExchange #14
+ WholeStageCodegen (23)
+ 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] #15
+ WholeStageCodegen (22)
+ 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_sold_date_sk,d_date_sk]
+ Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ BroadcastHashJoin [ss_item_sk,ss_item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #5
+ BroadcastExchange #16
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_week_seq,d_date_sk]
+ Subquery #6
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_week_seq]
+ CometFilter [d_year,d_moy,d_dom]
+ CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
+ InputAdapter
+ ReusedExchange [ss_item_sk] #3
+ InputAdapter
+ ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12
+ InputAdapter
+ ReusedExchange [d_date_sk] #16
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt
new file mode 100644
index 000000000..c4772ea80
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt
@@ -0,0 +1,170 @@
+== Physical Plan ==
+TakeOrderedAndProject (23)
++- * HashAggregate (22)
+ +- Exchange (21)
+ +- * ColumnarToRow (20)
+ +- CometHashAggregate (19)
+ +- CometProject (18)
+ +- CometBroadcastHashJoin (17)
+ :- CometProject (12)
+ : +- CometBroadcastHashJoin (11)
+ : :- CometProject (7)
+ : : +- CometBroadcastHashJoin (6)
+ : : :- CometFilter (2)
+ : : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : : +- CometBroadcastExchange (5)
+ : : +- CometFilter (4)
+ : : +- CometScan parquet spark_catalog.default.customer (3)
+ : +- CometBroadcastExchange (10)
+ : +- CometFilter (9)
+ : +- CometScan parquet spark_catalog.default.customer_address (8)
+ +- CometBroadcastExchange (16)
+ +- CometProject (15)
+ +- CometFilter (14)
+ +- CometScan parquet spark_catalog.default.date_dim (13)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)]
+PushedFilters: [IsNotNull(cs_bill_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3]
+Condition : isnotnull(cs_bill_customer_sk#1)
+
+(3) Scan parquet spark_catalog.default.customer
+Output [2]: [c_customer_sk#5, c_current_addr_sk#6]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [c_customer_sk#5, c_current_addr_sk#6]
+Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6))
+
+(5) CometBroadcastExchange
+Input [2]: [c_customer_sk#5, c_current_addr_sk#6]
+Arguments: [c_customer_sk#5, c_current_addr_sk#6]
+
+(6) CometBroadcastHashJoin
+Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3]
+Right output [2]: [c_customer_sk#5, c_current_addr_sk#6]
+Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight
+
+(7) CometProject
+Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6]
+Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6]
+
+(8) Scan parquet spark_catalog.default.customer_address
+Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9]
+Condition : isnotnull(ca_address_sk#7)
+
+(10) CometBroadcastExchange
+Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9]
+Arguments: [ca_address_sk#7, ca_state#8, ca_zip#9]
+
+(11) CometBroadcastHashJoin
+Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6]
+Right output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9]
+Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#8 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight
+
+(12) CometProject
+Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#8, ca_zip#9]
+Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9]
+
+(13) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#10, d_year#11, d_qoy#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(14) CometFilter
+Input [3]: [d_date_sk#10, d_year#11, d_qoy#12]
+Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2)) AND (d_year#11 = 2001)) AND isnotnull(d_date_sk#10))
+
+(15) CometProject
+Input [3]: [d_date_sk#10, d_year#11, d_qoy#12]
+Arguments: [d_date_sk#10], [d_date_sk#10]
+
+(16) CometBroadcastExchange
+Input [1]: [d_date_sk#10]
+Arguments: [d_date_sk#10]
+
+(17) CometBroadcastHashJoin
+Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9]
+Right output [1]: [d_date_sk#10]
+Arguments: [cs_sold_date_sk#3], [d_date_sk#10], Inner, BuildRight
+
+(18) CometProject
+Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9, d_date_sk#10]
+Arguments: [cs_sales_price#2, ca_zip#9], [cs_sales_price#2, ca_zip#9]
+
+(19) CometHashAggregate
+Input [2]: [cs_sales_price#2, ca_zip#9]
+Keys [1]: [ca_zip#9]
+Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))]
+
+(20) ColumnarToRow [codegen id : 1]
+Input [2]: [ca_zip#9, sum#13]
+
+(21) Exchange
+Input [2]: [ca_zip#9, sum#13]
+Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(22) HashAggregate [codegen id : 2]
+Input [2]: [ca_zip#9, sum#13]
+Keys [1]: [ca_zip#9]
+Functions [1]: [sum(UnscaledValue(cs_sales_price#2))]
+Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#14]
+Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#14,17,2) AS sum(cs_sales_price)#15]
+
+(23) TakeOrderedAndProject
+Input [2]: [ca_zip#9, sum(cs_sales_price)#15]
+Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#15]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4
+BroadcastExchange (28)
++- * ColumnarToRow (27)
+ +- CometProject (26)
+ +- CometFilter (25)
+ +- CometScan parquet spark_catalog.default.date_dim (24)
+
+
+(24) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#10, d_year#11, d_qoy#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(25) CometFilter
+Input [3]: [d_date_sk#10, d_year#11, d_qoy#12]
+Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2)) AND (d_year#11 = 2001)) AND isnotnull(d_date_sk#10))
+
+(26) CometProject
+Input [3]: [d_date_sk#10, d_year#11, d_qoy#12]
+Arguments: [d_date_sk#10], [d_date_sk#10]
+
+(27) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#10]
+
+(28) BroadcastExchange
+Input [1]: [d_date_sk#10]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt
new file mode 100644
index 000000000..a03346372
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt
@@ -0,0 +1,35 @@
+TakeOrderedAndProject [ca_zip,sum(cs_sales_price)]
+ WholeStageCodegen (2)
+ HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum]
+ InputAdapter
+ Exchange [ca_zip] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ca_zip,cs_sales_price]
+ CometProject [cs_sales_price,ca_zip]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [cs_sales_price,cs_sold_date_sk,ca_zip]
+ CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price]
+ CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometFilter [cs_bill_customer_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ CometBroadcastExchange #3
+ CometFilter [c_customer_sk,c_current_addr_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
+ CometBroadcastExchange #4
+ CometFilter [ca_address_sk]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip]
+ CometBroadcastExchange #5
+ CometProject [d_date_sk]
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt
new file mode 100644
index 000000000..ccec341ad
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt
@@ -0,0 +1,260 @@
+== Physical Plan ==
+* HashAggregate (45)
++- Exchange (44)
+ +- * HashAggregate (43)
+ +- * HashAggregate (42)
+ +- * HashAggregate (41)
+ +- * Project (40)
+ +- * BroadcastHashJoin Inner BuildRight (39)
+ :- * Project (33)
+ : +- * BroadcastHashJoin Inner BuildRight (32)
+ : :- * Project (26)
+ : : +- * BroadcastHashJoin Inner BuildRight (25)
+ : : :- * SortMergeJoin LeftAnti (19)
+ : : : :- * Project (13)
+ : : : : +- * SortMergeJoin LeftSemi (12)
+ : : : : :- * Sort (6)
+ : : : : : +- Exchange (5)
+ : : : : : +- * ColumnarToRow (4)
+ : : : : : +- CometProject (3)
+ : : : : : +- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : : : : +- * Sort (11)
+ : : : : +- Exchange (10)
+ : : : : +- * ColumnarToRow (9)
+ : : : : +- CometProject (8)
+ : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7)
+ : : : +- * Sort (18)
+ : : : +- Exchange (17)
+ : : : +- * ColumnarToRow (16)
+ : : : +- CometProject (15)
+ : : : +- CometScan parquet spark_catalog.default.catalog_returns (14)
+ : : +- BroadcastExchange (24)
+ : : +- * ColumnarToRow (23)
+ : : +- CometProject (22)
+ : : +- CometFilter (21)
+ : : +- CometScan parquet spark_catalog.default.date_dim (20)
+ : +- BroadcastExchange (31)
+ : +- * ColumnarToRow (30)
+ : +- CometProject (29)
+ : +- CometFilter (28)
+ : +- CometScan parquet spark_catalog.default.customer_address (27)
+ +- BroadcastExchange (38)
+ +- * ColumnarToRow (37)
+ +- CometProject (36)
+ +- CometFilter (35)
+ +- CometScan parquet spark_catalog.default.call_center (34)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/catalog_sales]
+PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8]
+Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3))
+
+(3) CometProject
+Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8]
+Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+
+(4) ColumnarToRow [codegen id : 1]
+Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+
+(5) Exchange
+Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(6) Sort [codegen id : 2]
+Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0
+
+(7) Scan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/catalog_sales]
+ReadSchema: struct
+
+(8) CometProject
+Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11]
+Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10]
+
+(9) ColumnarToRow [codegen id : 3]
+Input [2]: [cs_warehouse_sk#9, cs_order_number#10]
+
+(10) Exchange
+Input [2]: [cs_warehouse_sk#9, cs_order_number#10]
+Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(11) Sort [codegen id : 4]
+Input [2]: [cs_warehouse_sk#9, cs_order_number#10]
+Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0
+
+(12) SortMergeJoin [codegen id : 5]
+Left keys [1]: [cs_order_number#5]
+Right keys [1]: [cs_order_number#10]
+Join type: LeftSemi
+Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9)
+
+(13) Project [codegen id : 5]
+Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+
+(14) Scan parquet spark_catalog.default.catalog_returns
+Output [2]: [cr_order_number#12, cr_returned_date_sk#13]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/catalog_returns]
+ReadSchema: struct
+
+(15) CometProject
+Input [2]: [cr_order_number#12, cr_returned_date_sk#13]
+Arguments: [cr_order_number#12], [cr_order_number#12]
+
+(16) ColumnarToRow [codegen id : 6]
+Input [1]: [cr_order_number#12]
+
+(17) Exchange
+Input [1]: [cr_order_number#12]
+Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(18) Sort [codegen id : 7]
+Input [1]: [cr_order_number#12]
+Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0
+
+(19) SortMergeJoin [codegen id : 11]
+Left keys [1]: [cs_order_number#5]
+Right keys [1]: [cr_order_number#12]
+Join type: LeftAnti
+Join condition: None
+
+(20) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#14, d_date#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(21) CometFilter
+Input [2]: [d_date_sk#14, d_date#15]
+Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14))
+
+(22) CometProject
+Input [2]: [d_date_sk#14, d_date#15]
+Arguments: [d_date_sk#14], [d_date_sk#14]
+
+(23) ColumnarToRow [codegen id : 8]
+Input [1]: [d_date_sk#14]
+
+(24) BroadcastExchange
+Input [1]: [d_date_sk#14]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
+
+(25) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [cs_ship_date_sk#1]
+Right keys [1]: [d_date_sk#14]
+Join type: Inner
+Join condition: None
+
+(26) Project [codegen id : 11]
+Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14]
+
+(27) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#16, ca_state#17]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(28) CometFilter
+Input [2]: [ca_address_sk#16, ca_state#17]
+Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16))
+
+(29) CometProject
+Input [2]: [ca_address_sk#16, ca_state#17]
+Arguments: [ca_address_sk#16], [ca_address_sk#16]
+
+(30) ColumnarToRow [codegen id : 9]
+Input [1]: [ca_address_sk#16]
+
+(31) BroadcastExchange
+Input [1]: [ca_address_sk#16]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5]
+
+(32) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [cs_ship_addr_sk#2]
+Right keys [1]: [ca_address_sk#16]
+Join type: Inner
+Join condition: None
+
+(33) Project [codegen id : 11]
+Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16]
+
+(34) Scan parquet spark_catalog.default.call_center
+Output [2]: [cc_call_center_sk#18, cc_county#19]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/call_center]
+PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)]
+ReadSchema: struct
+
+(35) CometFilter
+Input [2]: [cc_call_center_sk#18, cc_county#19]
+Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18))
+
+(36) CometProject
+Input [2]: [cc_call_center_sk#18, cc_county#19]
+Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18]
+
+(37) ColumnarToRow [codegen id : 10]
+Input [1]: [cc_call_center_sk#18]
+
+(38) BroadcastExchange
+Input [1]: [cc_call_center_sk#18]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
+
+(39) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [cs_call_center_sk#3]
+Right keys [1]: [cc_call_center_sk#18]
+Join type: Inner
+Join condition: None
+
+(40) Project [codegen id : 11]
+Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18]
+
+(41) HashAggregate [codegen id : 11]
+Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7]
+Keys [1]: [cs_order_number#5]
+Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))]
+Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21]
+Results [3]: [cs_order_number#5, sum#22, sum#23]
+
+(42) HashAggregate [codegen id : 11]
+Input [3]: [cs_order_number#5, sum#22, sum#23]
+Keys [1]: [cs_order_number#5]
+Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))]
+Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21]
+Results [3]: [cs_order_number#5, sum#22, sum#23]
+
+(43) HashAggregate [codegen id : 11]
+Input [3]: [cs_order_number#5, sum#22, sum#23]
+Keys: []
+Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)]
+Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24]
+Results [3]: [sum#22, sum#23, count#25]
+
+(44) Exchange
+Input [3]: [sum#22, sum#23, count#25]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7]
+
+(45) HashAggregate [codegen id : 12]
+Input [3]: [sum#22, sum#23, count#25]
+Keys: []
+Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)]
+Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24]
+Results [3]: [count(cs_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#20,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#21,17,2) AS total net profit #28]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt
new file mode 100644
index 000000000..a55c182be
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt
@@ -0,0 +1,74 @@
+WholeStageCodegen (12)
+ HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count]
+ InputAdapter
+ Exchange #1
+ WholeStageCodegen (11)
+ HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count]
+ HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum]
+ HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum]
+ Project [cs_order_number,cs_ext_ship_cost,cs_net_profit]
+ BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
+ BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk]
+ Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
+ BroadcastHashJoin [cs_ship_date_sk,d_date_sk]
+ SortMergeJoin [cs_order_number,cr_order_number]
+ InputAdapter
+ WholeStageCodegen (5)
+ Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
+ SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk]
+ InputAdapter
+ WholeStageCodegen (2)
+ Sort [cs_order_number]
+ InputAdapter
+ Exchange [cs_order_number] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
+ CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [cs_order_number]
+ InputAdapter
+ Exchange [cs_order_number] #3
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometProject [cs_warehouse_sk,cs_order_number]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk]
+ InputAdapter
+ WholeStageCodegen (7)
+ Sort [cr_order_number]
+ InputAdapter
+ Exchange [cr_order_number] #4
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometProject [cr_order_number]
+ CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (8)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (9)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ca_address_sk]
+ CometFilter [ca_state,ca_address_sk]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (10)
+ ColumnarToRow
+ InputAdapter
+ CometProject [cc_call_center_sk]
+ CometFilter [cc_county,cc_call_center_sk]
+ CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt
new file mode 100644
index 000000000..8d9edc0a0
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt
@@ -0,0 +1,311 @@
+== Physical Plan ==
+TakeOrderedAndProject (42)
++- * HashAggregate (41)
+ +- Exchange (40)
+ +- * ColumnarToRow (39)
+ +- CometHashAggregate (38)
+ +- CometProject (37)
+ +- CometBroadcastHashJoin (36)
+ :- CometProject (32)
+ : +- CometBroadcastHashJoin (31)
+ : :- CometProject (27)
+ : : +- CometBroadcastHashJoin (26)
+ : : :- CometProject (24)
+ : : : +- CometBroadcastHashJoin (23)
+ : : : :- CometProject (18)
+ : : : : +- CometBroadcastHashJoin (17)
+ : : : : :- CometProject (12)
+ : : : : : +- CometBroadcastHashJoin (11)
+ : : : : : :- CometProject (7)
+ : : : : : : +- CometBroadcastHashJoin (6)
+ : : : : : : :- CometFilter (2)
+ : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : : : +- CometBroadcastExchange (5)
+ : : : : : : +- CometFilter (4)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3)
+ : : : : : +- CometBroadcastExchange (10)
+ : : : : : +- CometFilter (9)
+ : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8)
+ : : : : +- CometBroadcastExchange (16)
+ : : : : +- CometProject (15)
+ : : : : +- CometFilter (14)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (13)
+ : : : +- CometBroadcastExchange (22)
+ : : : +- CometProject (21)
+ : : : +- CometFilter (20)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (19)
+ : : +- ReusedExchange (25)
+ : +- CometBroadcastExchange (30)
+ : +- CometFilter (29)
+ : +- CometScan parquet spark_catalog.default.store (28)
+ +- CometBroadcastExchange (35)
+ +- CometFilter (34)
+ +- CometScan parquet spark_catalog.default.item (33)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)]
+PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
+Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3))
+
+(3) Scan parquet spark_catalog.default.store_returns
+Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)]
+PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10))
+
+(5) CometBroadcastExchange
+Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+
+(6) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
+Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight
+
+(7) CometProject
+Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12]
+
+(8) Scan parquet spark_catalog.default.catalog_sales
+Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)]
+PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15))
+
+(10) CometBroadcastExchange
+Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+
+(11) CometBroadcastHashJoin
+Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12]
+Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight
+
+(12) CometProject
+Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+
+(13) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#19, d_quarter_name#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(14) CometFilter
+Input [2]: [d_date_sk#19, d_quarter_name#20]
+Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND isnotnull(d_date_sk#19))
+
+(15) CometProject
+Input [2]: [d_date_sk#19, d_quarter_name#20]
+Arguments: [d_date_sk#19], [d_date_sk#19]
+
+(16) CometBroadcastExchange
+Input [1]: [d_date_sk#19]
+Arguments: [d_date_sk#19]
+
+(17) CometBroadcastHashJoin
+Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#19]
+Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight
+
+(18) CometProject
+Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+
+(19) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#21, d_quarter_name#22]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(20) CometFilter
+Input [2]: [d_date_sk#21, d_quarter_name#22]
+Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21))
+
+(21) CometProject
+Input [2]: [d_date_sk#21, d_quarter_name#22]
+Arguments: [d_date_sk#21], [d_date_sk#21]
+
+(22) CometBroadcastExchange
+Input [1]: [d_date_sk#21]
+Arguments: [d_date_sk#21]
+
+(23) CometBroadcastHashJoin
+Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#21]
+Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight
+
+(24) CometProject
+Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17]
+
+(25) ReusedExchange [Reuses operator id: 22]
+Output [1]: [d_date_sk#23]
+
+(26) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#23]
+Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight
+
+(27) CometProject
+Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16]
+
+(28) Scan parquet spark_catalog.default.store
+Output [2]: [s_store_sk#24, s_state#25]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(29) CometFilter
+Input [2]: [s_store_sk#24, s_state#25]
+Condition : isnotnull(s_store_sk#24)
+
+(30) CometBroadcastExchange
+Input [2]: [s_store_sk#24, s_state#25]
+Arguments: [s_store_sk#24, s_state#25]
+
+(31) CometBroadcastHashJoin
+Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16]
+Right output [2]: [s_store_sk#24, s_state#25]
+Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight
+
+(32) CometProject
+Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#25]
+Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25]
+
+(33) Scan parquet spark_catalog.default.item
+Output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(34) CometFilter
+Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28]
+Condition : isnotnull(i_item_sk#26)
+
+(35) CometBroadcastExchange
+Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28]
+Arguments: [i_item_sk#26, i_item_id#27, i_item_desc#28]
+
+(36) CometBroadcastHashJoin
+Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25]
+Right output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28]
+Arguments: [ss_item_sk#1], [i_item_sk#26], Inner, BuildRight
+
+(37) CometProject
+Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_sk#26, i_item_id#27, i_item_desc#28]
+Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28]
+
+(38) CometHashAggregate
+Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28]
+Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25]
+Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))]
+
+(39) ColumnarToRow [codegen id : 1]
+Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46]
+
+(40) Exchange
+Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46]
+Arguments: hashpartitioning(i_item_id#27, i_item_desc#28, s_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(41) HashAggregate [codegen id : 2]
+Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46]
+Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25]
+Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))]
+Aggregate Attributes [9]: [count(ss_quantity#5)#47, avg(ss_quantity#5)#48, stddev_samp(cast(ss_quantity#5 as double))#49, count(sr_return_quantity#11)#50, avg(sr_return_quantity#11)#51, stddev_samp(cast(sr_return_quantity#11 as double))#52, count(cs_quantity#16)#53, avg(cs_quantity#16)#54, stddev_samp(cast(cs_quantity#16 as double))#55]
+Results [15]: [i_item_id#27, i_item_desc#28, s_state#25, count(ss_quantity#5)#47 AS store_sales_quantitycount#56, avg(ss_quantity#5)#48 AS store_sales_quantityave#57, stddev_samp(cast(ss_quantity#5 as double))#49 AS store_sales_quantitystdev#58, (stddev_samp(cast(ss_quantity#5 as double))#49 / avg(ss_quantity#5)#48) AS store_sales_quantitycov#59, count(sr_return_quantity#11)#50 AS as_store_returns_quantitycount#60, avg(sr_return_quantity#11)#51 AS as_store_returns_quantityave#61, stddev_samp(cast(sr_return_quantity#11 as double))#52 AS as_store_returns_quantitystdev#62, (stddev_samp(cast(sr_return_quantity#11 as double))#52 / avg(sr_return_quantity#11)#51) AS store_returns_quantitycov#63, count(cs_quantity#16)#53 AS catalog_sales_quantitycount#64, avg(cs_quantity#16)#54 AS catalog_sales_quantityave#65, (stddev_samp(cast(cs_quantity#16 as double))#55 / avg(cs_quantity#16)#54) AS catalog_sales_quantitystdev#66, (stddev_samp(cast(cs_quantity#16 as double))#55 / avg(cs_quantity#16)#54) AS catalog_sales_quantitycov#67]
+
+(42) TakeOrderedAndProject
+Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#56, store_sales_quantityave#57, store_sales_quantitystdev#58, store_sales_quantitycov#59, as_store_returns_quantitycount#60, as_store_returns_quantityave#61, as_store_returns_quantitystdev#62, store_returns_quantitycov#63, catalog_sales_quantitycount#64, catalog_sales_quantityave#65, catalog_sales_quantitystdev#66, catalog_sales_quantitycov#67]
+Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#56, store_sales_quantityave#57, store_sales_quantitystdev#58, store_sales_quantitycov#59, as_store_returns_quantitycount#60, as_store_returns_quantityave#61, as_store_returns_quantitystdev#62, store_returns_quantitycov#63, catalog_sales_quantitycount#64, catalog_sales_quantityave#65, catalog_sales_quantitystdev#66, catalog_sales_quantitycov#67]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
+BroadcastExchange (47)
++- * ColumnarToRow (46)
+ +- CometProject (45)
+ +- CometFilter (44)
+ +- CometScan parquet spark_catalog.default.date_dim (43)
+
+
+(43) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#19, d_quarter_name#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(44) CometFilter
+Input [2]: [d_date_sk#19, d_quarter_name#20]
+Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND isnotnull(d_date_sk#19))
+
+(45) CometProject
+Input [2]: [d_date_sk#19, d_quarter_name#20]
+Arguments: [d_date_sk#19], [d_date_sk#19]
+
+(46) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#19]
+
+(47) BroadcastExchange
+Input [1]: [d_date_sk#19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13
+BroadcastExchange (52)
++- * ColumnarToRow (51)
+ +- CometProject (50)
+ +- CometFilter (49)
+ +- CometScan parquet spark_catalog.default.date_dim (48)
+
+
+(48) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#21, d_quarter_name#22]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(49) CometFilter
+Input [2]: [d_date_sk#21, d_quarter_name#22]
+Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21))
+
+(50) CometProject
+Input [2]: [d_date_sk#21, d_quarter_name#22]
+Arguments: [d_date_sk#21], [d_date_sk#21]
+
+(51) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#21]
+
+(52) BroadcastExchange
+Input [1]: [d_date_sk#21]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt
new file mode 100644
index 000000000..77aba376e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt
@@ -0,0 +1,63 @@
+TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov]
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2]
+ InputAdapter
+ Exchange [i_item_id,i_item_desc,s_state] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity]
+ CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state]
+ CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk]
+ CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk]
+ CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
+ CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_quarter_name,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
+ CometBroadcastExchange #3
+ CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #4
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_quarter_name,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
+ CometBroadcastExchange #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ CometBroadcastExchange #6
+ CometProject [d_date_sk]
+ CometFilter [d_quarter_name,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
+ CometBroadcastExchange #7
+ CometProject [d_date_sk]
+ CometFilter [d_quarter_name,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
+ ReusedExchange [d_date_sk] #7
+ CometBroadcastExchange #8
+ CometFilter [s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
+ CometBroadcastExchange #9
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt
new file mode 100644
index 000000000..c09ef1445
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt
@@ -0,0 +1,272 @@
+== Physical Plan ==
+TakeOrderedAndProject (41)
++- * HashAggregate (40)
+ +- Exchange (39)
+ +- * ColumnarToRow (38)
+ +- CometHashAggregate (37)
+ +- CometExpand (36)
+ +- CometProject (35)
+ +- CometBroadcastHashJoin (34)
+ :- CometProject (30)
+ : +- CometBroadcastHashJoin (29)
+ : :- CometProject (24)
+ : : +- CometBroadcastHashJoin (23)
+ : : :- CometProject (19)
+ : : : +- CometBroadcastHashJoin (18)
+ : : : :- CometProject (14)
+ : : : : +- CometBroadcastHashJoin (13)
+ : : : : :- CometProject (8)
+ : : : : : +- CometBroadcastHashJoin (7)
+ : : : : : :- CometFilter (2)
+ : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : : : : : +- CometBroadcastExchange (6)
+ : : : : : +- CometProject (5)
+ : : : : : +- CometFilter (4)
+ : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (3)
+ : : : : +- CometBroadcastExchange (12)
+ : : : : +- CometProject (11)
+ : : : : +- CometFilter (10)
+ : : : : +- CometScan parquet spark_catalog.default.customer (9)
+ : : : +- CometBroadcastExchange (17)
+ : : : +- CometFilter (16)
+ : : : +- CometScan parquet spark_catalog.default.customer_demographics (15)
+ : : +- CometBroadcastExchange (22)
+ : : +- CometFilter (21)
+ : : +- CometScan parquet spark_catalog.default.customer_address (20)
+ : +- CometBroadcastExchange (28)
+ : +- CometProject (27)
+ : +- CometFilter (26)
+ : +- CometScan parquet spark_catalog.default.date_dim (25)
+ +- CometBroadcastExchange (33)
+ +- CometFilter (32)
+ +- CometScan parquet spark_catalog.default.item (31)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)]
+PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9]
+Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3))
+
+(3) Scan parquet spark_catalog.default.customer_demographics
+Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14]
+Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11))
+
+(5) CometProject
+Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14]
+Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14]
+
+(6) CometBroadcastExchange
+Input [2]: [cd_demo_sk#11, cd_dep_count#14]
+Arguments: [cd_demo_sk#11, cd_dep_count#14]
+
+(7) CometBroadcastHashJoin
+Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9]
+Right output [2]: [cd_demo_sk#11, cd_dep_count#14]
+Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight
+
+(8) CometProject
+Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14]
+Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14]
+
+(9) Scan parquet spark_catalog.default.customer
+Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19]
+Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17))
+
+(11) CometProject
+Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19]
+Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19]
+
+(12) CometBroadcastExchange
+Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19]
+Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19]
+
+(13) CometBroadcastHashJoin
+Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14]
+Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19]
+Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight
+
+(14) CometProject
+Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19]
+Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19]
+
+(15) Scan parquet spark_catalog.default.customer_demographics
+Output [1]: [cd_demo_sk#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_demo_sk)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [1]: [cd_demo_sk#20]
+Condition : isnotnull(cd_demo_sk#20)
+
+(17) CometBroadcastExchange
+Input [1]: [cd_demo_sk#20]
+Arguments: [cd_demo_sk#20]
+
+(18) CometBroadcastHashJoin
+Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19]
+Right output [1]: [cd_demo_sk#20]
+Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight
+
+(19) CometProject
+Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20]
+Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19]
+
+(20) Scan parquet spark_catalog.default.customer_address
+Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [In(ca_state, [IN,MS,ND,NM,OK,VA]), IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(21) CometFilter
+Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24]
+Condition : (ca_state#23 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21))
+
+(22) CometBroadcastExchange
+Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24]
+Arguments: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24]
+
+(23) CometBroadcastHashJoin
+Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19]
+Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24]
+Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight
+
+(24) CometProject
+Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24]
+Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24]
+
+(25) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#25, d_year#26]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [d_date_sk#25, d_year#26]
+Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_sk#25))
+
+(27) CometProject
+Input [2]: [d_date_sk#25, d_year#26]
+Arguments: [d_date_sk#25], [d_date_sk#25]
+
+(28) CometBroadcastExchange
+Input [1]: [d_date_sk#25]
+Arguments: [d_date_sk#25]
+
+(29) CometBroadcastHashJoin
+Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24]
+Right output [1]: [d_date_sk#25]
+Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight
+
+(30) CometProject
+Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, d_date_sk#25]
+Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24]
+
+(31) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#27, i_item_id#28]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(32) CometFilter
+Input [2]: [i_item_sk#27, i_item_id#28]
+Condition : isnotnull(i_item_sk#27)
+
+(33) CometBroadcastExchange
+Input [2]: [i_item_sk#27, i_item_id#28]
+Arguments: [i_item_sk#27, i_item_id#28]
+
+(34) CometBroadcastHashJoin
+Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24]
+Right output [2]: [i_item_sk#27, i_item_id#28]
+Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight
+
+(35) CometProject
+Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28]
+Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22]
+
+(36) CometExpand
+Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22]
+Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33]
+
+(37) CometHashAggregate
+Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33]
+Keys [5]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33]
+Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))]
+
+(38) ColumnarToRow [codegen id : 1]
+Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47]
+
+(39) Exchange
+Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47]
+Arguments: hashpartitioning(i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(40) HashAggregate [codegen id : 2]
+Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47]
+Keys [5]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33]
+Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))]
+Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#48, avg(cast(cs_list_price#5 as decimal(12,2)))#49, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#50, avg(cast(cs_sales_price#6 as decimal(12,2)))#51, avg(cast(cs_net_profit#8 as decimal(12,2)))#52, avg(cast(c_birth_year#19 as decimal(12,2)))#53, avg(cast(cd_dep_count#14 as decimal(12,2)))#54]
+Results [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, avg(cast(cs_quantity#4 as decimal(12,2)))#48 AS agg1#55, avg(cast(cs_list_price#5 as decimal(12,2)))#49 AS agg2#56, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#50 AS agg3#57, avg(cast(cs_sales_price#6 as decimal(12,2)))#51 AS agg4#58, avg(cast(cs_net_profit#8 as decimal(12,2)))#52 AS agg5#59, avg(cast(c_birth_year#19 as decimal(12,2)))#53 AS agg6#60, avg(cast(cd_dep_count#14 as decimal(12,2)))#54 AS agg7#61]
+
+(41) TakeOrderedAndProject
+Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#55, agg2#56, agg3#57, agg4#58, agg5#59, agg6#60, agg7#61]
+Arguments: 100, [ca_country#30 ASC NULLS FIRST, ca_state#31 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, i_item_id#29 ASC NULLS FIRST], [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#55, agg2#56, agg3#57, agg4#58, agg5#59, agg6#60, agg7#61]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10
+BroadcastExchange (46)
++- * ColumnarToRow (45)
+ +- CometProject (44)
+ +- CometFilter (43)
+ +- CometScan parquet spark_catalog.default.date_dim (42)
+
+
+(42) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#25, d_year#26]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(43) CometFilter
+Input [2]: [d_date_sk#25, d_year#26]
+Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_sk#25))
+
+(44) CometProject
+Input [2]: [d_date_sk#25, d_year#26]
+Arguments: [d_date_sk#25], [d_date_sk#25]
+
+(45) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#25]
+
+(46) BroadcastExchange
+Input [1]: [d_date_sk#25]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt
new file mode 100644
index 000000000..3d101857b
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt
@@ -0,0 +1,53 @@
+TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count]
+ InputAdapter
+ Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count]
+ CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id]
+ CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county]
+ CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country]
+ CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year]
+ CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
+ CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count]
+ CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #3
+ CometProject [cd_demo_sk,cd_dep_count]
+ CometFilter [cd_gender,cd_education_status,cd_demo_sk]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
+ CometBroadcastExchange #4
+ CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
+ CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year]
+ CometBroadcastExchange #5
+ CometFilter [cd_demo_sk]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
+ CometBroadcastExchange #6
+ CometFilter [ca_state,ca_address_sk]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country]
+ CometBroadcastExchange #7
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #8
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt
new file mode 100644
index 000000000..a00474bee
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt
@@ -0,0 +1,200 @@
+== Physical Plan ==
+TakeOrderedAndProject (34)
++- * HashAggregate (33)
+ +- Exchange (32)
+ +- * ColumnarToRow (31)
+ +- CometHashAggregate (30)
+ +- CometProject (29)
+ +- CometBroadcastHashJoin (28)
+ :- CometProject (24)
+ : +- CometBroadcastHashJoin (23)
+ : :- CometProject (19)
+ : : +- CometBroadcastHashJoin (18)
+ : : :- CometProject (14)
+ : : : +- CometBroadcastHashJoin (13)
+ : : : :- CometProject (8)
+ : : : : +- CometBroadcastHashJoin (7)
+ : : : : :- CometProject (3)
+ : : : : : +- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.date_dim (1)
+ : : : : +- CometBroadcastExchange (6)
+ : : : : +- CometFilter (5)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (4)
+ : : : +- CometBroadcastExchange (12)
+ : : : +- CometProject (11)
+ : : : +- CometFilter (10)
+ : : : +- CometScan parquet spark_catalog.default.item (9)
+ : : +- CometBroadcastExchange (17)
+ : : +- CometFilter (16)
+ : : +- CometScan parquet spark_catalog.default.customer (15)
+ : +- CometBroadcastExchange (22)
+ : +- CometFilter (21)
+ : +- CometScan parquet spark_catalog.default.customer_address (20)
+ +- CometBroadcastExchange (27)
+ +- CometFilter (26)
+ +- CometScan parquet spark_catalog.default.store (25)
+
+
+(1) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#1, d_year#2, d_moy#3]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [d_date_sk#1, d_year#2, d_moy#3]
+Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1))
+
+(3) CometProject
+Input [3]: [d_date_sk#1, d_year#2, d_moy#3]
+Arguments: [d_date_sk#1], [d_date_sk#1]
+
+(4) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)]
+PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)]
+ReadSchema: struct
+
+(5) CometFilter
+Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8]
+Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6))
+
+(6) CometBroadcastExchange
+Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8]
+Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8]
+
+(7) CometBroadcastHashJoin
+Left output [1]: [d_date_sk#1]
+Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8]
+Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight
+
+(8) CometProject
+Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8]
+Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7]
+
+(9) Scan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14]
+Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9))
+
+(11) CometProject
+Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14]
+Arguments: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+
+(12) CometBroadcastExchange
+Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+Arguments: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+
+(13) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7]
+Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight
+
+(14) CometProject
+Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+
+(15) Scan parquet spark_catalog.default.customer
+Output [2]: [c_customer_sk#15, c_current_addr_sk#16]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [2]: [c_customer_sk#15, c_current_addr_sk#16]
+Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16))
+
+(17) CometBroadcastExchange
+Input [2]: [c_customer_sk#15, c_current_addr_sk#16]
+Arguments: [c_customer_sk#15, c_current_addr_sk#16]
+
+(18) CometBroadcastHashJoin
+Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+Right output [2]: [c_customer_sk#15, c_current_addr_sk#16]
+Arguments: [ss_customer_sk#5], [c_customer_sk#15], Inner, BuildRight
+
+(19) CometProject
+Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_customer_sk#15, c_current_addr_sk#16]
+Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16]
+
+(20) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#17, ca_zip#18]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)]
+ReadSchema: struct
+
+(21) CometFilter
+Input [2]: [ca_address_sk#17, ca_zip#18]
+Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18))
+
+(22) CometBroadcastExchange
+Input [2]: [ca_address_sk#17, ca_zip#18]
+Arguments: [ca_address_sk#17, ca_zip#18]
+
+(23) CometBroadcastHashJoin
+Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16]
+Right output [2]: [ca_address_sk#17, ca_zip#18]
+Arguments: [c_current_addr_sk#16], [ca_address_sk#17], Inner, BuildRight
+
+(24) CometProject
+Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16, ca_address_sk#17, ca_zip#18]
+Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18]
+
+(25) Scan parquet spark_catalog.default.store
+Output [2]: [s_store_sk#19, s_zip#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [s_store_sk#19, s_zip#20]
+Condition : (isnotnull(s_zip#20) AND isnotnull(s_store_sk#19))
+
+(27) CometBroadcastExchange
+Input [2]: [s_store_sk#19, s_zip#20]
+Arguments: [s_store_sk#19, s_zip#20]
+
+(28) CometBroadcastHashJoin
+Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18]
+Right output [2]: [s_store_sk#19, s_zip#20]
+Arguments: [ss_store_sk#6], [s_store_sk#19], Inner, NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#20, 1, 5)), BuildRight
+
+(29) CometProject
+Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18, s_store_sk#19, s_zip#20]
+Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+
+(30) CometHashAggregate
+Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13]
+Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13]
+Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))]
+
+(31) ColumnarToRow [codegen id : 1]
+Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#21]
+
+(32) Exchange
+Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#21]
+Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(33) HashAggregate [codegen id : 2]
+Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#21]
+Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13]
+Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#22]
+Results [5]: [i_brand_id#10 AS brand_id#23, i_brand#11 AS brand#24, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#22,17,2) AS ext_price#25]
+
+(34) TakeOrderedAndProject
+Input [5]: [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25]
+Arguments: 100, [ext_price#25 DESC NULLS LAST, brand#24 ASC NULLS FIRST, brand_id#23 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt
new file mode 100644
index 000000000..18a69bcb4
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt
@@ -0,0 +1,38 @@
+TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact]
+ WholeStageCodegen (2)
+ HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum]
+ InputAdapter
+ Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price]
+ CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact]
+ CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip]
+ CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip]
+ CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk]
+ CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price]
+ CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #2
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
+ CometBroadcastExchange #3
+ CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact]
+ CometFilter [i_manager_id,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id]
+ CometBroadcastExchange #4
+ CometFilter [c_customer_sk,c_current_addr_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
+ CometBroadcastExchange #5
+ CometFilter [ca_address_sk,ca_zip]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip]
+ CometBroadcastExchange #6
+ CometFilter [s_zip,s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_zip]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt
new file mode 100644
index 000000000..8565313c9
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt
@@ -0,0 +1,203 @@
+== Physical Plan ==
+* Sort (35)
++- Exchange (34)
+ +- * Project (33)
+ +- * BroadcastHashJoin Inner BuildRight (32)
+ :- * Project (21)
+ : +- * BroadcastHashJoin Inner BuildRight (20)
+ : :- * HashAggregate (14)
+ : : +- Exchange (13)
+ : : +- * ColumnarToRow (12)
+ : : +- CometHashAggregate (11)
+ : : +- CometProject (10)
+ : : +- CometBroadcastHashJoin (9)
+ : : :- CometUnion (5)
+ : : : :- CometProject (2)
+ : : : : +- CometScan parquet spark_catalog.default.web_sales (1)
+ : : : +- CometProject (4)
+ : : : +- CometScan parquet spark_catalog.default.catalog_sales (3)
+ : : +- CometBroadcastExchange (8)
+ : : +- CometFilter (7)
+ : : +- CometScan parquet spark_catalog.default.date_dim (6)
+ : +- BroadcastExchange (19)
+ : +- * ColumnarToRow (18)
+ : +- CometProject (17)
+ : +- CometFilter (16)
+ : +- CometScan parquet spark_catalog.default.date_dim (15)
+ +- BroadcastExchange (31)
+ +- * Project (30)
+ +- * BroadcastHashJoin Inner BuildRight (29)
+ :- * HashAggregate (23)
+ : +- ReusedExchange (22)
+ +- BroadcastExchange (28)
+ +- * ColumnarToRow (27)
+ +- CometProject (26)
+ +- CometFilter (25)
+ +- CometScan parquet spark_catalog.default.date_dim (24)
+
+
+(1) Scan parquet spark_catalog.default.web_sales
+Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#2)]
+ReadSchema: struct
+
+(2) CometProject
+Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2]
+Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4]
+
+(3) Scan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#6)]
+ReadSchema: struct
+
+(4) CometProject
+Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6]
+Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8]
+
+(5) CometUnion
+Child 0 Input [2]: [sold_date_sk#3, sales_price#4]
+Child 1 Input [2]: [sold_date_sk#7, sales_price#8]
+
+(6) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)]
+ReadSchema: struct
+
+(7) CometFilter
+Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11]
+Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10))
+
+(8) CometBroadcastExchange
+Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11]
+Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#11]
+
+(9) CometBroadcastHashJoin
+Left output [2]: [sold_date_sk#3, sales_price#4]
+Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11]
+Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight
+
+(10) CometProject
+Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11]
+Arguments: [sales_price#4, d_week_seq#10, d_day_name#11], [sales_price#4, d_week_seq#10, d_day_name#11]
+
+(11) CometHashAggregate
+Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11]
+Keys [1]: [d_week_seq#10]
+Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))]
+
+(12) ColumnarToRow [codegen id : 1]
+Input [8]: [d_week_seq#10, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18]
+
+(13) Exchange
+Input [8]: [d_week_seq#10, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18]
+Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(14) HashAggregate [codegen id : 6]
+Input [8]: [d_week_seq#10, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18]
+Keys [1]: [d_week_seq#10]
+Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))]
+Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#25]
+Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#19,17,2) AS sun_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#20,17,2) AS mon_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#21,17,2) AS tue_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#22,17,2) AS wed_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#23,17,2) AS thu_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#24,17,2) AS fri_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#25,17,2) AS sat_sales#32]
+
+(15) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_week_seq#33, d_year#34]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [2]: [d_week_seq#33, d_year#34]
+Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2001)) AND isnotnull(d_week_seq#33))
+
+(17) CometProject
+Input [2]: [d_week_seq#33, d_year#34]
+Arguments: [d_week_seq#33], [d_week_seq#33]
+
+(18) ColumnarToRow [codegen id : 2]
+Input [1]: [d_week_seq#33]
+
+(19) BroadcastExchange
+Input [1]: [d_week_seq#33]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+(20) BroadcastHashJoin [codegen id : 6]
+Left keys [1]: [d_week_seq#10]
+Right keys [1]: [d_week_seq#33]
+Join type: Inner
+Join condition: None
+
+(21) Project [codegen id : 6]
+Output [8]: [d_week_seq#10 AS d_week_seq1#35, sun_sales#26 AS sun_sales1#36, mon_sales#27 AS mon_sales1#37, tue_sales#28 AS tue_sales1#38, wed_sales#29 AS wed_sales1#39, thu_sales#30 AS thu_sales1#40, fri_sales#31 AS fri_sales1#41, sat_sales#32 AS sat_sales1#42]
+Input [9]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32, d_week_seq#33]
+
+(22) ReusedExchange [Reuses operator id: 13]
+Output [8]: [d_week_seq#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50]
+
+(23) HashAggregate [codegen id : 5]
+Input [8]: [d_week_seq#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50]
+Keys [1]: [d_week_seq#43]
+Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#51 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#51 = Saturday ) THEN sales_price#4 END))]
+Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#51 = Sunday ) THEN sales_price#4 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Monday ) THEN sales_price#4 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Tuesday ) THEN sales_price#4 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Wednesday) THEN sales_price#4 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Thursday ) THEN sales_price#4 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Friday ) THEN sales_price#4 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#51 = Saturday ) THEN sales_price#4 END))#25]
+Results [8]: [d_week_seq#43, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Sunday ) THEN sales_price#4 END))#19,17,2) AS sun_sales#52, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Monday ) THEN sales_price#4 END))#20,17,2) AS mon_sales#53, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Tuesday ) THEN sales_price#4 END))#21,17,2) AS tue_sales#54, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Wednesday) THEN sales_price#4 END))#22,17,2) AS wed_sales#55, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Thursday ) THEN sales_price#4 END))#23,17,2) AS thu_sales#56, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Friday ) THEN sales_price#4 END))#24,17,2) AS fri_sales#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#51 = Saturday ) THEN sales_price#4 END))#25,17,2) AS sat_sales#58]
+
+(24) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_week_seq#59, d_year#60]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)]
+ReadSchema: struct
+
+(25) CometFilter
+Input [2]: [d_week_seq#59, d_year#60]
+Condition : ((isnotnull(d_year#60) AND (d_year#60 = 2002)) AND isnotnull(d_week_seq#59))
+
+(26) CometProject
+Input [2]: [d_week_seq#59, d_year#60]
+Arguments: [d_week_seq#59], [d_week_seq#59]
+
+(27) ColumnarToRow [codegen id : 4]
+Input [1]: [d_week_seq#59]
+
+(28) BroadcastExchange
+Input [1]: [d_week_seq#59]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+(29) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [d_week_seq#43]
+Right keys [1]: [d_week_seq#59]
+Join type: Inner
+Join condition: None
+
+(30) Project [codegen id : 5]
+Output [8]: [d_week_seq#43 AS d_week_seq2#61, sun_sales#52 AS sun_sales2#62, mon_sales#53 AS mon_sales2#63, tue_sales#54 AS tue_sales2#64, wed_sales#55 AS wed_sales2#65, thu_sales#56 AS thu_sales2#66, fri_sales#57 AS fri_sales2#67, sat_sales#58 AS sat_sales2#68]
+Input [9]: [d_week_seq#43, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, d_week_seq#59]
+
+(31) BroadcastExchange
+Input [8]: [d_week_seq2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68]
+Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=4]
+
+(32) BroadcastHashJoin [codegen id : 6]
+Left keys [1]: [d_week_seq1#35]
+Right keys [1]: [(d_week_seq2#61 - 53)]
+Join type: Inner
+Join condition: None
+
+(33) Project [codegen id : 6]
+Output [8]: [d_week_seq1#35, round((sun_sales1#36 / sun_sales2#62), 2) AS round((sun_sales1 / sun_sales2), 2)#69, round((mon_sales1#37 / mon_sales2#63), 2) AS round((mon_sales1 / mon_sales2), 2)#70, round((tue_sales1#38 / tue_sales2#64), 2) AS round((tue_sales1 / tue_sales2), 2)#71, round((wed_sales1#39 / wed_sales2#65), 2) AS round((wed_sales1 / wed_sales2), 2)#72, round((thu_sales1#40 / thu_sales2#66), 2) AS round((thu_sales1 / thu_sales2), 2)#73, round((fri_sales1#41 / fri_sales2#67), 2) AS round((fri_sales1 / fri_sales2), 2)#74, round((sat_sales1#42 / sat_sales2#68), 2) AS round((sat_sales1 / sat_sales2), 2)#75]
+Input [16]: [d_week_seq1#35, sun_sales1#36, mon_sales1#37, tue_sales1#38, wed_sales1#39, thu_sales1#40, fri_sales1#41, sat_sales1#42, d_week_seq2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68]
+
+(34) Exchange
+Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#69, round((mon_sales1 / mon_sales2), 2)#70, round((tue_sales1 / tue_sales2), 2)#71, round((wed_sales1 / wed_sales2), 2)#72, round((thu_sales1 / thu_sales2), 2)#73, round((fri_sales1 / fri_sales2), 2)#74, round((sat_sales1 / sat_sales2), 2)#75]
+Arguments: rangepartitioning(d_week_seq1#35 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+
+(35) Sort [codegen id : 7]
+Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#69, round((mon_sales1 / mon_sales2), 2)#70, round((tue_sales1 / tue_sales2), 2)#71, round((wed_sales1 / wed_sales2), 2)#72, round((thu_sales1 / thu_sales2), 2)#73, round((fri_sales1 / fri_sales2), 2)#74, round((sat_sales1 / sat_sales2), 2)#75]
+Arguments: [d_week_seq1#35 ASC NULLS FIRST], true, 0
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt
new file mode 100644
index 000000000..c7999d981
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt
@@ -0,0 +1,50 @@
+WholeStageCodegen (7)
+ Sort [d_week_seq1]
+ InputAdapter
+ Exchange [d_week_seq1] #1
+ WholeStageCodegen (6)
+ Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2]
+ BroadcastHashJoin [d_week_seq1,d_week_seq2]
+ Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales]
+ BroadcastHashJoin [d_week_seq,d_week_seq]
+ HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum]
+ InputAdapter
+ Exchange [d_week_seq] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [d_week_seq,d_day_name,sales_price]
+ CometProject [sales_price,d_week_seq,d_day_name]
+ CometBroadcastHashJoin [sold_date_sk,d_date_sk]
+ CometUnion
+ CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price]
+ CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk]
+ CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk]
+ CometBroadcastExchange #3
+ CometFilter [d_date_sk,d_week_seq]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_week_seq]
+ CometFilter [d_year,d_week_seq]
+ CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (5)
+ Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales]
+ BroadcastHashJoin [d_week_seq,d_week_seq]
+ HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum]
+ InputAdapter
+ ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (4)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_week_seq]
+ CometFilter [d_year,d_week_seq]
+ CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt
new file mode 100644
index 000000000..7c29ff218
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt
@@ -0,0 +1,161 @@
+== Physical Plan ==
+TakeOrderedAndProject (22)
++- * Project (21)
+ +- Window (20)
+ +- * Sort (19)
+ +- Exchange (18)
+ +- * HashAggregate (17)
+ +- Exchange (16)
+ +- * ColumnarToRow (15)
+ +- CometHashAggregate (14)
+ +- CometProject (13)
+ +- CometBroadcastHashJoin (12)
+ :- CometProject (7)
+ : +- CometBroadcastHashJoin (6)
+ : :- CometFilter (2)
+ : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : +- CometBroadcastExchange (5)
+ : +- CometFilter (4)
+ : +- CometScan parquet spark_catalog.default.item (3)
+ +- CometBroadcastExchange (11)
+ +- CometProject (10)
+ +- CometFilter (9)
+ +- CometScan parquet spark_catalog.default.date_dim (8)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3]
+Condition : isnotnull(cs_item_sk#1)
+
+(3) Scan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5))
+
+(5) CometBroadcastExchange
+Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+
+(6) CometBroadcastHashJoin
+Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3]
+Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight
+
+(7) CometProject
+Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+
+(8) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#11, d_date#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [2]: [d_date_sk#11, d_date#12]
+Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11))
+
+(10) CometProject
+Input [2]: [d_date_sk#11, d_date#12]
+Arguments: [d_date_sk#11], [d_date_sk#11]
+
+(11) CometBroadcastExchange
+Input [1]: [d_date_sk#11]
+Arguments: [d_date_sk#11]
+
+(12) CometBroadcastHashJoin
+Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Right output [1]: [d_date_sk#11]
+Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight
+
+(13) CometProject
+Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11]
+Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+
+(14) CometHashAggregate
+Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10]
+Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8]
+Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))]
+
+(15) ColumnarToRow [codegen id : 1]
+Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13]
+
+(16) Exchange
+Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13]
+Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(17) HashAggregate [codegen id : 2]
+Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13]
+Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8]
+Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))]
+Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14]
+Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6]
+
+(18) Exchange
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6]
+Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(19) Sort [codegen id : 3]
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6]
+Arguments: [i_class#9 ASC NULLS FIRST], false, 0
+
+(20) Window
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6]
+Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9]
+
+(21) Project [codegen id : 4]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6]
+Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17]
+
+(22) TakeOrderedAndProject
+Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6]
+Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4
+BroadcastExchange (27)
++- * ColumnarToRow (26)
+ +- CometProject (25)
+ +- CometFilter (24)
+ +- CometScan parquet spark_catalog.default.date_dim (23)
+
+
+(23) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#11, d_date#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(24) CometFilter
+Input [2]: [d_date_sk#11, d_date#12]
+Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11))
+
+(25) CometProject
+Input [2]: [d_date_sk#11, d_date#12]
+Arguments: [d_date_sk#11], [d_date_sk#11]
+
+(26) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#11]
+
+(27) BroadcastExchange
+Input [1]: [d_date_sk#11]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt
new file mode 100644
index 000000000..d805e3868
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt
@@ -0,0 +1,38 @@
+TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue]
+ WholeStageCodegen (4)
+ Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id]
+ InputAdapter
+ Window [_w0,i_class]
+ WholeStageCodegen (3)
+ Sort [i_class]
+ InputAdapter
+ Exchange [i_class] #1
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum]
+ InputAdapter
+ Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price]
+ CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometFilter [cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
+ CometBroadcastExchange #4
+ CometFilter [i_category,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometBroadcastExchange #5
+ CometProject [d_date_sk]
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt
new file mode 100644
index 000000000..21c979264
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt
@@ -0,0 +1,170 @@
+== Physical Plan ==
+TakeOrderedAndProject (24)
++- * Filter (23)
+ +- * HashAggregate (22)
+ +- Exchange (21)
+ +- * ColumnarToRow (20)
+ +- CometHashAggregate (19)
+ +- CometProject (18)
+ +- CometBroadcastHashJoin (17)
+ :- CometProject (13)
+ : +- CometBroadcastHashJoin (12)
+ : :- CometProject (7)
+ : : +- CometBroadcastHashJoin (6)
+ : : :- CometFilter (2)
+ : : : +- CometScan parquet spark_catalog.default.inventory (1)
+ : : +- CometBroadcastExchange (5)
+ : : +- CometFilter (4)
+ : : +- CometScan parquet spark_catalog.default.warehouse (3)
+ : +- CometBroadcastExchange (11)
+ : +- CometProject (10)
+ : +- CometFilter (9)
+ : +- CometScan parquet spark_catalog.default.item (8)
+ +- CometBroadcastExchange (16)
+ +- CometFilter (15)
+ +- CometScan parquet spark_catalog.default.date_dim (14)
+
+
+(1) Scan parquet spark_catalog.default.inventory
+Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)]
+PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4]
+Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1))
+
+(3) Scan parquet spark_catalog.default.warehouse
+Output [2]: [w_warehouse_sk#6, w_warehouse_name#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/warehouse]
+PushedFilters: [IsNotNull(w_warehouse_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [w_warehouse_sk#6, w_warehouse_name#7]
+Condition : isnotnull(w_warehouse_sk#6)
+
+(5) CometBroadcastExchange
+Input [2]: [w_warehouse_sk#6, w_warehouse_name#7]
+Arguments: [w_warehouse_sk#6, w_warehouse_name#7]
+
+(6) CometBroadcastHashJoin
+Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4]
+Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7]
+Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight
+
+(7) CometProject
+Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7]
+Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7]
+
+(8) Scan parquet spark_catalog.default.item
+Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10]
+Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8))
+
+(10) CometProject
+Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10]
+Arguments: [i_item_sk#8, i_item_id#9], [i_item_sk#8, i_item_id#9]
+
+(11) CometBroadcastExchange
+Input [2]: [i_item_sk#8, i_item_id#9]
+Arguments: [i_item_sk#8, i_item_id#9]
+
+(12) CometBroadcastHashJoin
+Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7]
+Right output [2]: [i_item_sk#8, i_item_id#9]
+Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight
+
+(13) CometProject
+Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#9]
+Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9]
+
+(14) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#11, d_date#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(15) CometFilter
+Input [2]: [d_date_sk#11, d_date#12]
+Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11))
+
+(16) CometBroadcastExchange
+Input [2]: [d_date_sk#11, d_date#12]
+Arguments: [d_date_sk#11, d_date#12]
+
+(17) CometBroadcastHashJoin
+Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9]
+Right output [2]: [d_date_sk#11, d_date#12]
+Arguments: [inv_date_sk#4], [d_date_sk#11], Inner, BuildRight
+
+(18) CometProject
+Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9, d_date_sk#11, d_date#12]
+Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12]
+
+(19) CometHashAggregate
+Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12]
+Keys [2]: [w_warehouse_name#7, i_item_id#9]
+Functions [2]: [partial_sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)]
+
+(20) ColumnarToRow [codegen id : 1]
+Input [4]: [w_warehouse_name#7, i_item_id#9, sum#13, sum#14]
+
+(21) Exchange
+Input [4]: [w_warehouse_name#7, i_item_id#9, sum#13, sum#14]
+Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(22) HashAggregate [codegen id : 2]
+Input [4]: [w_warehouse_name#7, i_item_id#9, sum#13, sum#14]
+Keys [2]: [w_warehouse_name#7, i_item_id#9]
+Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)]
+Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#15, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#16]
+Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#15 AS inv_before#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#16 AS inv_after#18]
+
+(23) Filter [codegen id : 2]
+Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18]
+Condition : (CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) >= 0.666667) END AND CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) <= 1.5) END)
+
+(24) TakeOrderedAndProject
+Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18]
+Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5
+BroadcastExchange (28)
++- * ColumnarToRow (27)
+ +- CometFilter (26)
+ +- CometScan parquet spark_catalog.default.date_dim (25)
+
+
+(25) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#11, d_date#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [d_date_sk#11, d_date#12]
+Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11))
+
+(27) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#11, d_date#12]
+
+(28) BroadcastExchange
+Input [2]: [d_date_sk#11, d_date#12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt
new file mode 100644
index 000000000..52bd7a85e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt
@@ -0,0 +1,35 @@
+TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after]
+ WholeStageCodegen (2)
+ Filter [inv_before,inv_after]
+ HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum]
+ InputAdapter
+ Exchange [w_warehouse_name,i_item_id] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand]
+ CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date]
+ CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id]
+ CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name]
+ CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometFilter [inv_warehouse_sk,inv_item_sk]
+ CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
+ CometBroadcastExchange #3
+ CometFilter [w_warehouse_sk]
+ CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
+ CometBroadcastExchange #4
+ CometProject [i_item_sk,i_item_id]
+ CometFilter [i_current_price,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price]
+ CometBroadcastExchange #5
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt
new file mode 100644
index 000000000..df6a80179
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt
@@ -0,0 +1,175 @@
+== Physical Plan ==
+TakeOrderedAndProject (24)
++- * HashAggregate (23)
+ +- Exchange (22)
+ +- * ColumnarToRow (21)
+ +- CometHashAggregate (20)
+ +- CometExpand (19)
+ +- CometProject (18)
+ +- CometBroadcastHashJoin (17)
+ :- CometProject (13)
+ : +- CometBroadcastHashJoin (12)
+ : :- CometProject (8)
+ : : +- CometBroadcastHashJoin (7)
+ : : :- CometFilter (2)
+ : : : +- CometScan parquet spark_catalog.default.inventory (1)
+ : : +- CometBroadcastExchange (6)
+ : : +- CometProject (5)
+ : : +- CometFilter (4)
+ : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : +- CometBroadcastExchange (11)
+ : +- CometFilter (10)
+ : +- CometScan parquet spark_catalog.default.item (9)
+ +- CometBroadcastExchange (16)
+ +- CometFilter (15)
+ +- CometScan parquet spark_catalog.default.warehouse (14)
+
+
+(1) Scan parquet spark_catalog.default.inventory
+Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)]
+PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4]
+Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2))
+
+(3) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#6, d_month_seq#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [d_date_sk#6, d_month_seq#7]
+Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6))
+
+(5) CometProject
+Input [2]: [d_date_sk#6, d_month_seq#7]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(6) CometBroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: [d_date_sk#6]
+
+(7) CometBroadcastHashJoin
+Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4]
+Right output [1]: [d_date_sk#6]
+Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight
+
+(8) CometProject
+Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6]
+Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3]
+
+(9) Scan parquet spark_catalog.default.item
+Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+Condition : isnotnull(i_item_sk#8)
+
+(11) CometBroadcastExchange
+Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+Arguments: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+
+(12) CometBroadcastHashJoin
+Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3]
+Right output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight
+
+(13) CometProject
+Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+
+(14) Scan parquet spark_catalog.default.warehouse
+Output [1]: [w_warehouse_sk#13]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/warehouse]
+PushedFilters: [IsNotNull(w_warehouse_sk)]
+ReadSchema: struct
+
+(15) CometFilter
+Input [1]: [w_warehouse_sk#13]
+Condition : isnotnull(w_warehouse_sk#13)
+
+(16) CometBroadcastExchange
+Input [1]: [w_warehouse_sk#13]
+Arguments: [w_warehouse_sk#13]
+
+(17) CometBroadcastHashJoin
+Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12]
+Right output [1]: [w_warehouse_sk#13]
+Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#13], Inner, BuildRight
+
+(18) CometProject
+Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#13]
+Arguments: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11]
+
+(19) CometExpand
+Input [5]: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11]
+Arguments: [[inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, null, 1], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, null, null, 3], [inv_quantity_on_hand#3, i_product_name#12, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18]
+
+(20) CometHashAggregate
+Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18]
+Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18]
+Functions [1]: [partial_avg(inv_quantity_on_hand#3)]
+
+(21) ColumnarToRow [codegen id : 1]
+Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#19, count#20]
+
+(22) Exchange
+Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#19, count#20]
+Arguments: hashpartitioning(i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(23) HashAggregate [codegen id : 2]
+Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#19, count#20]
+Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18]
+Functions [1]: [avg(inv_quantity_on_hand#3)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21]
+Results [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, avg(inv_quantity_on_hand#3)#21 AS qoh#22]
+
+(24) TakeOrderedAndProject
+Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#22]
+Arguments: 100, [qoh#22 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#22]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5
+BroadcastExchange (29)
++- * ColumnarToRow (28)
+ +- CometProject (27)
+ +- CometFilter (26)
+ +- CometScan parquet spark_catalog.default.date_dim (25)
+
+
+(25) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#6, d_month_seq#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [d_date_sk#6, d_month_seq#7]
+Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6))
+
+(27) CometProject
+Input [2]: [d_date_sk#6, d_month_seq#7]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(28) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#6]
+
+(29) BroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt
new file mode 100644
index 000000000..bda583c17
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt
@@ -0,0 +1,36 @@
+TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
+ WholeStageCodegen (2)
+ HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count]
+ InputAdapter
+ Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand]
+ CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id]
+ CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category]
+ CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
+ CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand]
+ CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_month_seq,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
+ CometBroadcastExchange #3
+ CometProject [d_date_sk]
+ CometFilter [d_month_seq,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
+ CometBroadcastExchange #4
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
+ CometBroadcastExchange #5
+ CometFilter [w_warehouse_sk]
+ CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt
new file mode 100644
index 000000000..9c7a61130
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt
@@ -0,0 +1,589 @@
+== Physical Plan ==
+* HashAggregate (67)
++- Exchange (66)
+ +- * HashAggregate (65)
+ +- Union (64)
+ :- * Project (46)
+ : +- * BroadcastHashJoin Inner BuildRight (45)
+ : :- * Project (43)
+ : : +- * SortMergeJoin LeftSemi (42)
+ : : :- * Sort (26)
+ : : : +- Exchange (25)
+ : : : +- * Project (24)
+ : : : +- * BroadcastHashJoin LeftSemi BuildRight (23)
+ : : : :- * ColumnarToRow (2)
+ : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : : : +- BroadcastExchange (22)
+ : : : +- * Project (21)
+ : : : +- * Filter (20)
+ : : : +- * HashAggregate (19)
+ : : : +- Exchange (18)
+ : : : +- * ColumnarToRow (17)
+ : : : +- CometHashAggregate (16)
+ : : : +- CometProject (15)
+ : : : +- CometBroadcastHashJoin (14)
+ : : : :- CometProject (10)
+ : : : : +- CometBroadcastHashJoin (9)
+ : : : : :- CometFilter (4)
+ : : : : : +- CometScan parquet spark_catalog.default.store_sales (3)
+ : : : : +- CometBroadcastExchange (8)
+ : : : : +- CometProject (7)
+ : : : : +- CometFilter (6)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (5)
+ : : : +- CometBroadcastExchange (13)
+ : : : +- CometFilter (12)
+ : : : +- CometScan parquet spark_catalog.default.item (11)
+ : : +- * Sort (41)
+ : : +- * Project (40)
+ : : +- * Filter (39)
+ : : +- * HashAggregate (38)
+ : : +- Exchange (37)
+ : : +- * ColumnarToRow (36)
+ : : +- CometHashAggregate (35)
+ : : +- CometProject (34)
+ : : +- CometBroadcastHashJoin (33)
+ : : :- CometProject (29)
+ : : : +- CometFilter (28)
+ : : : +- CometScan parquet spark_catalog.default.store_sales (27)
+ : : +- CometBroadcastExchange (32)
+ : : +- CometFilter (31)
+ : : +- CometScan parquet spark_catalog.default.customer (30)
+ : +- ReusedExchange (44)
+ +- * Project (63)
+ +- * BroadcastHashJoin Inner BuildRight (62)
+ :- * Project (60)
+ : +- * SortMergeJoin LeftSemi (59)
+ : :- * Sort (53)
+ : : +- Exchange (52)
+ : : +- * Project (51)
+ : : +- * BroadcastHashJoin LeftSemi BuildRight (50)
+ : : :- * ColumnarToRow (48)
+ : : : +- CometScan parquet spark_catalog.default.web_sales (47)
+ : : +- ReusedExchange (49)
+ : +- * Sort (58)
+ : +- * Project (57)
+ : +- * Filter (56)
+ : +- * HashAggregate (55)
+ : +- ReusedExchange (54)
+ +- ReusedExchange (61)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)]
+ReadSchema: struct
+
+(2) ColumnarToRow [codegen id : 3]
+Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+
+(3) Scan parquet spark_catalog.default.store_sales
+Output [2]: [ss_item_sk#7, ss_sold_date_sk#8]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [ss_item_sk#7, ss_sold_date_sk#8]
+Condition : isnotnull(ss_item_sk#7)
+
+(5) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#10, d_date#11, d_year#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(6) CometFilter
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+
+(7) CometProject
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11]
+
+(8) CometBroadcastExchange
+Input [2]: [d_date_sk#10, d_date#11]
+Arguments: [d_date_sk#10, d_date#11]
+
+(9) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8]
+Right output [2]: [d_date_sk#10, d_date#11]
+Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight
+
+(10) CometProject
+Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11]
+Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11]
+
+(11) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#13, i_item_desc#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(12) CometFilter
+Input [2]: [i_item_sk#13, i_item_desc#14]
+Condition : isnotnull(i_item_sk#13)
+
+(13) CometBroadcastExchange
+Input [2]: [i_item_sk#13, i_item_desc#14]
+Arguments: [i_item_sk#13, i_item_desc#14]
+
+(14) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#7, d_date#11]
+Right output [2]: [i_item_sk#13, i_item_desc#14]
+Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight
+
+(15) CometProject
+Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14]
+Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15]
+
+(16) CometHashAggregate
+Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15]
+Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11]
+Functions [1]: [partial_count(1)]
+
+(17) ColumnarToRow [codegen id : 1]
+Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16]
+
+(18) Exchange
+Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16]
+Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(19) HashAggregate [codegen id : 2]
+Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16]
+Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11]
+Functions [1]: [count(1)]
+Aggregate Attributes [1]: [count(1)#17]
+Results [2]: [i_item_sk#13 AS item_sk#18, count(1)#17 AS cnt#19]
+
+(20) Filter [codegen id : 2]
+Input [2]: [item_sk#18, cnt#19]
+Condition : (cnt#19 > 4)
+
+(21) Project [codegen id : 2]
+Output [1]: [item_sk#18]
+Input [2]: [item_sk#18, cnt#19]
+
+(22) BroadcastExchange
+Input [1]: [item_sk#18]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+(23) BroadcastHashJoin [codegen id : 3]
+Left keys [1]: [cs_item_sk#2]
+Right keys [1]: [item_sk#18]
+Join type: LeftSemi
+Join condition: None
+
+(24) Project [codegen id : 3]
+Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+
+(25) Exchange
+Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(26) Sort [codegen id : 4]
+Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0
+
+(27) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(28) CometFilter
+Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23]
+Condition : isnotnull(ss_customer_sk#20)
+
+(29) CometProject
+Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23]
+Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22]
+
+(30) Scan parquet spark_catalog.default.customer
+Output [1]: [c_customer_sk#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk)]
+ReadSchema: struct
+
+(31) CometFilter
+Input [1]: [c_customer_sk#24]
+Condition : isnotnull(c_customer_sk#24)
+
+(32) CometBroadcastExchange
+Input [1]: [c_customer_sk#24]
+Arguments: [c_customer_sk#24]
+
+(33) CometBroadcastHashJoin
+Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22]
+Right output [1]: [c_customer_sk#24]
+Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight
+
+(34) CometProject
+Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
+Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
+
+(35) CometHashAggregate
+Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
+Keys [1]: [c_customer_sk#24]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+
+(36) ColumnarToRow [codegen id : 5]
+Input [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+
+(37) Exchange
+Input [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+
+(38) HashAggregate [codegen id : 6]
+Input [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+Keys [1]: [c_customer_sk#24]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28]
+
+(39) Filter [codegen id : 6]
+Input [2]: [c_customer_sk#24, ssales#28]
+Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30])))
+
+(40) Project [codegen id : 6]
+Output [1]: [c_customer_sk#24]
+Input [2]: [c_customer_sk#24, ssales#28]
+
+(41) Sort [codegen id : 6]
+Input [1]: [c_customer_sk#24]
+Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0
+
+(42) SortMergeJoin [codegen id : 8]
+Left keys [1]: [cs_bill_customer_sk#1]
+Right keys [1]: [c_customer_sk#24]
+Join type: LeftSemi
+Join condition: None
+
+(43) Project [codegen id : 8]
+Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+
+(44) ReusedExchange [Reuses operator id: 72]
+Output [1]: [d_date_sk#31]
+
+(45) BroadcastHashJoin [codegen id : 8]
+Left keys [1]: [cs_sold_date_sk#5]
+Right keys [1]: [d_date_sk#31]
+Join type: Inner
+Join condition: None
+
+(46) Project [codegen id : 8]
+Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32]
+Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#31]
+
+(47) Scan parquet spark_catalog.default.web_sales
+Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)]
+ReadSchema: struct
+
+(48) ColumnarToRow [codegen id : 11]
+Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+
+(49) ReusedExchange [Reuses operator id: 22]
+Output [1]: [item_sk#39]
+
+(50) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [ws_item_sk#33]
+Right keys [1]: [item_sk#39]
+Join type: LeftSemi
+Join condition: None
+
+(51) Project [codegen id : 11]
+Output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+
+(52) Exchange
+Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+
+(53) Sort [codegen id : 12]
+Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+Arguments: [ws_bill_customer_sk#34 ASC NULLS FIRST], false, 0
+
+(54) ReusedExchange [Reuses operator id: 37]
+Output [3]: [c_customer_sk#40, sum#41, isEmpty#42]
+
+(55) HashAggregate [codegen id : 14]
+Input [3]: [c_customer_sk#40, sum#41, isEmpty#42]
+Keys [1]: [c_customer_sk#40]
+Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))#27]
+Results [2]: [c_customer_sk#40, sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))#27 AS ssales#45]
+
+(56) Filter [codegen id : 14]
+Input [2]: [c_customer_sk#40, ssales#45]
+Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30])))
+
+(57) Project [codegen id : 14]
+Output [1]: [c_customer_sk#40]
+Input [2]: [c_customer_sk#40, ssales#45]
+
+(58) Sort [codegen id : 14]
+Input [1]: [c_customer_sk#40]
+Arguments: [c_customer_sk#40 ASC NULLS FIRST], false, 0
+
+(59) SortMergeJoin [codegen id : 16]
+Left keys [1]: [ws_bill_customer_sk#34]
+Right keys [1]: [c_customer_sk#40]
+Join type: LeftSemi
+Join condition: None
+
+(60) Project [codegen id : 16]
+Output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37]
+
+(61) ReusedExchange [Reuses operator id: 72]
+Output [1]: [d_date_sk#46]
+
+(62) BroadcastHashJoin [codegen id : 16]
+Left keys [1]: [ws_sold_date_sk#37]
+Right keys [1]: [d_date_sk#46]
+Join type: Inner
+Join condition: None
+
+(63) Project [codegen id : 16]
+Output [1]: [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#47]
+Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#46]
+
+(64) Union
+
+(65) HashAggregate [codegen id : 17]
+Input [1]: [sales#32]
+Keys: []
+Functions [1]: [partial_sum(sales#32)]
+Aggregate Attributes [2]: [sum#48, isEmpty#49]
+Results [2]: [sum#50, isEmpty#51]
+
+(66) Exchange
+Input [2]: [sum#50, isEmpty#51]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6]
+
+(67) HashAggregate [codegen id : 18]
+Input [2]: [sum#50, isEmpty#51]
+Keys: []
+Functions [1]: [sum(sales#32)]
+Aggregate Attributes [1]: [sum(sales#32)#52]
+Results [1]: [sum(sales#32)#52 AS sum(sales)#53]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6
+BroadcastExchange (72)
++- * ColumnarToRow (71)
+ +- CometProject (70)
+ +- CometFilter (69)
+ +- CometScan parquet spark_catalog.default.date_dim (68)
+
+
+(68) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#31, d_year#54, d_moy#55]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(69) CometFilter
+Input [3]: [d_date_sk#31, d_year#54, d_moy#55]
+Condition : ((((isnotnull(d_year#54) AND isnotnull(d_moy#55)) AND (d_year#54 = 2000)) AND (d_moy#55 = 2)) AND isnotnull(d_date_sk#31))
+
+(70) CometProject
+Input [3]: [d_date_sk#31, d_year#54, d_moy#55]
+Arguments: [d_date_sk#31], [d_date_sk#31]
+
+(71) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#31]
+
+(72) BroadcastExchange
+Input [1]: [d_date_sk#31]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
+
+Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9
+BroadcastExchange (77)
++- * ColumnarToRow (76)
+ +- CometProject (75)
+ +- CometFilter (74)
+ +- CometScan parquet spark_catalog.default.date_dim (73)
+
+
+(73) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#10, d_date#11, d_year#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(74) CometFilter
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+
+(75) CometProject
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11]
+
+(76) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#10, d_date#11]
+
+(77) BroadcastExchange
+Input [2]: [d_date_sk#10, d_date#11]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8]
+
+Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#29, [id=#30]
+* HashAggregate (95)
++- Exchange (94)
+ +- * HashAggregate (93)
+ +- * HashAggregate (92)
+ +- Exchange (91)
+ +- * ColumnarToRow (90)
+ +- CometHashAggregate (89)
+ +- CometProject (88)
+ +- CometBroadcastHashJoin (87)
+ :- CometProject (82)
+ : +- CometBroadcastHashJoin (81)
+ : :- CometFilter (79)
+ : : +- CometScan parquet spark_catalog.default.store_sales (78)
+ : +- ReusedExchange (80)
+ +- CometBroadcastExchange (86)
+ +- CometProject (85)
+ +- CometFilter (84)
+ +- CometScan parquet spark_catalog.default.date_dim (83)
+
+
+(78) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)]
+PushedFilters: [IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(79) CometFilter
+Input [4]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59]
+Condition : isnotnull(ss_customer_sk#56)
+
+(80) ReusedExchange [Reuses operator id: 32]
+Output [1]: [c_customer_sk#61]
+
+(81) CometBroadcastHashJoin
+Left output [4]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59]
+Right output [1]: [c_customer_sk#61]
+Arguments: [ss_customer_sk#56], [c_customer_sk#61], Inner, BuildRight
+
+(82) CometProject
+Input [5]: [ss_customer_sk#56, ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61]
+Arguments: [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61], [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61]
+
+(83) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#62, d_year#63]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(84) CometFilter
+Input [2]: [d_date_sk#62, d_year#63]
+Condition : (d_year#63 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#62))
+
+(85) CometProject
+Input [2]: [d_date_sk#62, d_year#63]
+Arguments: [d_date_sk#62], [d_date_sk#62]
+
+(86) CometBroadcastExchange
+Input [1]: [d_date_sk#62]
+Arguments: [d_date_sk#62]
+
+(87) CometBroadcastHashJoin
+Left output [4]: [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61]
+Right output [1]: [d_date_sk#62]
+Arguments: [ss_sold_date_sk#59], [d_date_sk#62], Inner, BuildRight
+
+(88) CometProject
+Input [5]: [ss_quantity#57, ss_sales_price#58, ss_sold_date_sk#59, c_customer_sk#61, d_date_sk#62]
+Arguments: [ss_quantity#57, ss_sales_price#58, c_customer_sk#61], [ss_quantity#57, ss_sales_price#58, c_customer_sk#61]
+
+(89) CometHashAggregate
+Input [3]: [ss_quantity#57, ss_sales_price#58, c_customer_sk#61]
+Keys [1]: [c_customer_sk#61]
+Functions [1]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))]
+
+(90) ColumnarToRow [codegen id : 1]
+Input [3]: [c_customer_sk#61, sum#64, isEmpty#65]
+
+(91) Exchange
+Input [3]: [c_customer_sk#61, sum#64, isEmpty#65]
+Arguments: hashpartitioning(c_customer_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=9]
+
+(92) HashAggregate [codegen id : 2]
+Input [3]: [c_customer_sk#61, sum#64, isEmpty#65]
+Keys [1]: [c_customer_sk#61]
+Functions [1]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))#66]
+Results [1]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_sales_price#58))#66 AS csales#67]
+
+(93) HashAggregate [codegen id : 2]
+Input [1]: [csales#67]
+Keys: []
+Functions [1]: [partial_max(csales#67)]
+Aggregate Attributes [1]: [max#68]
+Results [1]: [max#69]
+
+(94) Exchange
+Input [1]: [max#69]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10]
+
+(95) HashAggregate [codegen id : 3]
+Input [1]: [max#69]
+Keys: []
+Functions [1]: [max(csales#67)]
+Aggregate Attributes [1]: [max(csales#67)#70]
+Results [1]: [max(csales#67)#70 AS tpcds_cmax#71]
+
+Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60
+BroadcastExchange (100)
++- * ColumnarToRow (99)
+ +- CometProject (98)
+ +- CometFilter (97)
+ +- CometScan parquet spark_catalog.default.date_dim (96)
+
+
+(96) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#62, d_year#63]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(97) CometFilter
+Input [2]: [d_date_sk#62, d_year#63]
+Condition : (d_year#63 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#62))
+
+(98) CometProject
+Input [2]: [d_date_sk#62, d_year#63]
+Arguments: [d_date_sk#62], [d_date_sk#62]
+
+(99) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#62]
+
+(100) BroadcastExchange
+Input [1]: [d_date_sk#62]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11]
+
+Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6
+
+Subquery:6 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt
new file mode 100644
index 000000000..f818fd25f
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt
@@ -0,0 +1,150 @@
+WholeStageCodegen (18)
+ HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty]
+ InputAdapter
+ Exchange #1
+ WholeStageCodegen (17)
+ HashAggregate [sales] [sum,isEmpty,sum,isEmpty]
+ InputAdapter
+ Union
+ WholeStageCodegen (8)
+ Project [cs_quantity,cs_list_price]
+ BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ Project [cs_quantity,cs_list_price,cs_sold_date_sk]
+ SortMergeJoin [cs_bill_customer_sk,c_customer_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [cs_bill_customer_sk]
+ InputAdapter
+ Exchange [cs_bill_customer_sk] #2
+ WholeStageCodegen (3)
+ Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+ BroadcastHashJoin [cs_item_sk,item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (2)
+ Project [item_sk]
+ Filter [cnt]
+ HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count]
+ InputAdapter
+ Exchange [_groupingexpression,i_item_sk,d_date] #5
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,d_date]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #6
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk,d_date]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
+ CometBroadcastExchange #7
+ CometProject [d_date_sk,d_date]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
+ CometBroadcastExchange #8
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
+ InputAdapter
+ WholeStageCodegen (6)
+ Sort [c_customer_sk]
+ Project [c_customer_sk]
+ Filter [ssales]
+ Subquery #3
+ WholeStageCodegen (3)
+ HashAggregate [max] [max(csales),tpcds_cmax,max]
+ InputAdapter
+ Exchange #11
+ WholeStageCodegen (2)
+ HashAggregate [csales] [max,max]
+ HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty]
+ InputAdapter
+ Exchange [c_customer_sk] #12
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometProject [ss_quantity,ss_sales_price,c_customer_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometFilter [ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #4
+ BroadcastExchange #13
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ ReusedExchange [c_customer_sk] #10
+ CometBroadcastExchange #14
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
+ InputAdapter
+ Exchange [c_customer_sk] #9
+ WholeStageCodegen (5)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometProject [ss_quantity,ss_sales_price,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
+ CometFilter [ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+ CometBroadcastExchange #10
+ CometFilter [c_customer_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk]
+ InputAdapter
+ ReusedExchange [d_date_sk] #3
+ WholeStageCodegen (16)
+ Project [ws_quantity,ws_list_price]
+ BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ Project [ws_quantity,ws_list_price,ws_sold_date_sk]
+ SortMergeJoin [ws_bill_customer_sk,c_customer_sk]
+ InputAdapter
+ WholeStageCodegen (12)
+ Sort [ws_bill_customer_sk]
+ InputAdapter
+ Exchange [ws_bill_customer_sk] #15
+ WholeStageCodegen (11)
+ Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+ BroadcastHashJoin [ws_item_sk,item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ InputAdapter
+ ReusedExchange [item_sk] #4
+ InputAdapter
+ WholeStageCodegen (14)
+ Sort [c_customer_sk]
+ Project [c_customer_sk]
+ Filter [ssales]
+ ReusedSubquery [tpcds_cmax] #3
+ HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
+ InputAdapter
+ ReusedExchange [c_customer_sk,sum,isEmpty] #9
+ InputAdapter
+ ReusedExchange [d_date_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt
new file mode 100644
index 000000000..1b35c983a
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt
@@ -0,0 +1,713 @@
+== Physical Plan ==
+TakeOrderedAndProject (88)
++- Union (87)
+ :- * HashAggregate (63)
+ : +- Exchange (62)
+ : +- * HashAggregate (61)
+ : +- * Project (60)
+ : +- * BroadcastHashJoin Inner BuildRight (59)
+ : :- * Project (57)
+ : : +- * BroadcastHashJoin Inner BuildRight (56)
+ : : :- * SortMergeJoin LeftSemi (43)
+ : : : :- * Sort (27)
+ : : : : +- Exchange (26)
+ : : : : +- * Project (25)
+ : : : : +- * BroadcastHashJoin LeftSemi BuildRight (24)
+ : : : : :- * ColumnarToRow (3)
+ : : : : : +- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : : : : +- BroadcastExchange (23)
+ : : : : +- * Project (22)
+ : : : : +- * Filter (21)
+ : : : : +- * HashAggregate (20)
+ : : : : +- Exchange (19)
+ : : : : +- * ColumnarToRow (18)
+ : : : : +- CometHashAggregate (17)
+ : : : : +- CometProject (16)
+ : : : : +- CometBroadcastHashJoin (15)
+ : : : : :- CometProject (11)
+ : : : : : +- CometBroadcastHashJoin (10)
+ : : : : : :- CometFilter (5)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4)
+ : : : : : +- CometBroadcastExchange (9)
+ : : : : : +- CometProject (8)
+ : : : : : +- CometFilter (7)
+ : : : : : +- CometScan parquet spark_catalog.default.date_dim (6)
+ : : : : +- CometBroadcastExchange (14)
+ : : : : +- CometFilter (13)
+ : : : : +- CometScan parquet spark_catalog.default.item (12)
+ : : : +- * Sort (42)
+ : : : +- * Project (41)
+ : : : +- * Filter (40)
+ : : : +- * HashAggregate (39)
+ : : : +- Exchange (38)
+ : : : +- * ColumnarToRow (37)
+ : : : +- CometHashAggregate (36)
+ : : : +- CometProject (35)
+ : : : +- CometBroadcastHashJoin (34)
+ : : : :- CometProject (30)
+ : : : : +- CometFilter (29)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (28)
+ : : : +- CometBroadcastExchange (33)
+ : : : +- CometFilter (32)
+ : : : +- CometScan parquet spark_catalog.default.customer (31)
+ : : +- BroadcastExchange (55)
+ : : +- * SortMergeJoin LeftSemi (54)
+ : : :- * Sort (48)
+ : : : +- Exchange (47)
+ : : : +- * ColumnarToRow (46)
+ : : : +- CometFilter (45)
+ : : : +- CometScan parquet spark_catalog.default.customer (44)
+ : : +- * Sort (53)
+ : : +- * Project (52)
+ : : +- * Filter (51)
+ : : +- * HashAggregate (50)
+ : : +- ReusedExchange (49)
+ : +- ReusedExchange (58)
+ +- * HashAggregate (86)
+ +- Exchange (85)
+ +- * HashAggregate (84)
+ +- * Project (83)
+ +- * BroadcastHashJoin Inner BuildRight (82)
+ :- * Project (80)
+ : +- * BroadcastHashJoin Inner BuildRight (79)
+ : :- * SortMergeJoin LeftSemi (77)
+ : : :- * Sort (71)
+ : : : +- Exchange (70)
+ : : : +- * Project (69)
+ : : : +- * BroadcastHashJoin LeftSemi BuildRight (68)
+ : : : :- * ColumnarToRow (66)
+ : : : : +- CometFilter (65)
+ : : : : +- CometScan parquet spark_catalog.default.web_sales (64)
+ : : : +- ReusedExchange (67)
+ : : +- * Sort (76)
+ : : +- * Project (75)
+ : : +- * Filter (74)
+ : : +- * HashAggregate (73)
+ : : +- ReusedExchange (72)
+ : +- ReusedExchange (78)
+ +- ReusedExchange (81)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)]
+PushedFilters: [IsNotNull(cs_bill_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Condition : isnotnull(cs_bill_customer_sk#1)
+
+(3) ColumnarToRow [codegen id : 3]
+Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+
+(4) Scan parquet spark_catalog.default.store_sales
+Output [2]: [ss_item_sk#7, ss_sold_date_sk#8]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(5) CometFilter
+Input [2]: [ss_item_sk#7, ss_sold_date_sk#8]
+Condition : isnotnull(ss_item_sk#7)
+
+(6) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#10, d_date#11, d_year#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(7) CometFilter
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+
+(8) CometProject
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11]
+
+(9) CometBroadcastExchange
+Input [2]: [d_date_sk#10, d_date#11]
+Arguments: [d_date_sk#10, d_date#11]
+
+(10) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8]
+Right output [2]: [d_date_sk#10, d_date#11]
+Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight
+
+(11) CometProject
+Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11]
+Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11]
+
+(12) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#13, i_item_desc#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(13) CometFilter
+Input [2]: [i_item_sk#13, i_item_desc#14]
+Condition : isnotnull(i_item_sk#13)
+
+(14) CometBroadcastExchange
+Input [2]: [i_item_sk#13, i_item_desc#14]
+Arguments: [i_item_sk#13, i_item_desc#14]
+
+(15) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#7, d_date#11]
+Right output [2]: [i_item_sk#13, i_item_desc#14]
+Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight
+
+(16) CometProject
+Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14]
+Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15]
+
+(17) CometHashAggregate
+Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15]
+Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11]
+Functions [1]: [partial_count(1)]
+
+(18) ColumnarToRow [codegen id : 1]
+Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16]
+
+(19) Exchange
+Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16]
+Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(20) HashAggregate [codegen id : 2]
+Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16]
+Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11]
+Functions [1]: [count(1)]
+Aggregate Attributes [1]: [count(1)#17]
+Results [2]: [i_item_sk#13 AS item_sk#18, count(1)#17 AS cnt#19]
+
+(21) Filter [codegen id : 2]
+Input [2]: [item_sk#18, cnt#19]
+Condition : (cnt#19 > 4)
+
+(22) Project [codegen id : 2]
+Output [1]: [item_sk#18]
+Input [2]: [item_sk#18, cnt#19]
+
+(23) BroadcastExchange
+Input [1]: [item_sk#18]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+(24) BroadcastHashJoin [codegen id : 3]
+Left keys [1]: [cs_item_sk#2]
+Right keys [1]: [item_sk#18]
+Join type: LeftSemi
+Join condition: None
+
+(25) Project [codegen id : 3]
+Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+
+(26) Exchange
+Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(27) Sort [codegen id : 4]
+Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5]
+Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0
+
+(28) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(29) CometFilter
+Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23]
+Condition : isnotnull(ss_customer_sk#20)
+
+(30) CometProject
+Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23]
+Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22]
+
+(31) Scan parquet spark_catalog.default.customer
+Output [1]: [c_customer_sk#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk)]
+ReadSchema: struct
+
+(32) CometFilter
+Input [1]: [c_customer_sk#24]
+Condition : isnotnull(c_customer_sk#24)
+
+(33) CometBroadcastExchange
+Input [1]: [c_customer_sk#24]
+Arguments: [c_customer_sk#24]
+
+(34) CometBroadcastHashJoin
+Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22]
+Right output [1]: [c_customer_sk#24]
+Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight
+
+(35) CometProject
+Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
+Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
+
+(36) CometHashAggregate
+Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
+Keys [1]: [c_customer_sk#24]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+
+(37) ColumnarToRow [codegen id : 5]
+Input [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+
+(38) Exchange
+Input [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+
+(39) HashAggregate [codegen id : 6]
+Input [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+Keys [1]: [c_customer_sk#24]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28]
+
+(40) Filter [codegen id : 6]
+Input [2]: [c_customer_sk#24, ssales#28]
+Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30])))
+
+(41) Project [codegen id : 6]
+Output [1]: [c_customer_sk#24]
+Input [2]: [c_customer_sk#24, ssales#28]
+
+(42) Sort [codegen id : 6]
+Input [1]: [c_customer_sk#24]
+Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0
+
+(43) SortMergeJoin [codegen id : 13]
+Left keys [1]: [cs_bill_customer_sk#1]
+Right keys [1]: [c_customer_sk#24]
+Join type: LeftSemi
+Join condition: None
+
+(44) Scan parquet spark_catalog.default.customer
+Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk)]
+ReadSchema: struct
+
+(45) CometFilter
+Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33]
+Condition : isnotnull(c_customer_sk#31)
+
+(46) ColumnarToRow [codegen id : 7]
+Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33]
+
+(47) Exchange
+Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33]
+Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+
+(48) Sort [codegen id : 8]
+Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33]
+Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0
+
+(49) ReusedExchange [Reuses operator id: 38]
+Output [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+
+(50) HashAggregate [codegen id : 10]
+Input [3]: [c_customer_sk#24, sum#25, isEmpty#26]
+Keys [1]: [c_customer_sk#24]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28]
+
+(51) Filter [codegen id : 10]
+Input [2]: [c_customer_sk#24, ssales#28]
+Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30])))
+
+(52) Project [codegen id : 10]
+Output [1]: [c_customer_sk#24]
+Input [2]: [c_customer_sk#24, ssales#28]
+
+(53) Sort [codegen id : 10]
+Input [1]: [c_customer_sk#24]
+Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0
+
+(54) SortMergeJoin [codegen id : 11]
+Left keys [1]: [c_customer_sk#31]
+Right keys [1]: [c_customer_sk#24]
+Join type: LeftSemi
+Join condition: None
+
+(55) BroadcastExchange
+Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6]
+
+(56) BroadcastHashJoin [codegen id : 13]
+Left keys [1]: [cs_bill_customer_sk#1]
+Right keys [1]: [c_customer_sk#31]
+Join type: Inner
+Join condition: None
+
+(57) Project [codegen id : 13]
+Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33]
+Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#31, c_first_name#32, c_last_name#33]
+
+(58) ReusedExchange [Reuses operator id: 93]
+Output [1]: [d_date_sk#34]
+
+(59) BroadcastHashJoin [codegen id : 13]
+Left keys [1]: [cs_sold_date_sk#5]
+Right keys [1]: [d_date_sk#34]
+Join type: Inner
+Join condition: None
+
+(60) Project [codegen id : 13]
+Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33]
+Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34]
+
+(61) HashAggregate [codegen id : 13]
+Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33]
+Keys [2]: [c_last_name#33, c_first_name#32]
+Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))]
+Aggregate Attributes [2]: [sum#35, isEmpty#36]
+Results [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38]
+
+(62) Exchange
+Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38]
+Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+
+(63) HashAggregate [codegen id : 14]
+Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38]
+Keys [2]: [c_last_name#33, c_first_name#32]
+Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))]
+Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#39]
+Results [3]: [c_last_name#33, c_first_name#32, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#39 AS sales#40]
+
+(64) Scan parquet spark_catalog.default.web_sales
+Output [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)]
+PushedFilters: [IsNotNull(ws_bill_customer_sk)]
+ReadSchema: struct
+
+(65) CometFilter
+Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45]
+Condition : isnotnull(ws_bill_customer_sk#42)
+
+(66) ColumnarToRow [codegen id : 17]
+Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45]
+
+(67) ReusedExchange [Reuses operator id: 23]
+Output [1]: [item_sk#47]
+
+(68) BroadcastHashJoin [codegen id : 17]
+Left keys [1]: [ws_item_sk#41]
+Right keys [1]: [item_sk#47]
+Join type: LeftSemi
+Join condition: None
+
+(69) Project [codegen id : 17]
+Output [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45]
+Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45]
+
+(70) Exchange
+Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45]
+Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=8]
+
+(71) Sort [codegen id : 18]
+Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45]
+Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0
+
+(72) ReusedExchange [Reuses operator id: 38]
+Output [3]: [c_customer_sk#48, sum#49, isEmpty#50]
+
+(73) HashAggregate [codegen id : 20]
+Input [3]: [c_customer_sk#48, sum#49, isEmpty#50]
+Keys [1]: [c_customer_sk#48]
+Functions [1]: [sum((cast(ss_quantity#51 as decimal(10,0)) * ss_sales_price#52))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#51 as decimal(10,0)) * ss_sales_price#52))#27]
+Results [2]: [c_customer_sk#48, sum((cast(ss_quantity#51 as decimal(10,0)) * ss_sales_price#52))#27 AS ssales#53]
+
+(74) Filter [codegen id : 20]
+Input [2]: [c_customer_sk#48, ssales#53]
+Condition : (isnotnull(ssales#53) AND (cast(ssales#53 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30])))
+
+(75) Project [codegen id : 20]
+Output [1]: [c_customer_sk#48]
+Input [2]: [c_customer_sk#48, ssales#53]
+
+(76) Sort [codegen id : 20]
+Input [1]: [c_customer_sk#48]
+Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0
+
+(77) SortMergeJoin [codegen id : 27]
+Left keys [1]: [ws_bill_customer_sk#42]
+Right keys [1]: [c_customer_sk#48]
+Join type: LeftSemi
+Join condition: None
+
+(78) ReusedExchange [Reuses operator id: 55]
+Output [3]: [c_customer_sk#54, c_first_name#55, c_last_name#56]
+
+(79) BroadcastHashJoin [codegen id : 27]
+Left keys [1]: [ws_bill_customer_sk#42]
+Right keys [1]: [c_customer_sk#54]
+Join type: Inner
+Join condition: None
+
+(80) Project [codegen id : 27]
+Output [5]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_first_name#55, c_last_name#56]
+Input [7]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_customer_sk#54, c_first_name#55, c_last_name#56]
+
+(81) ReusedExchange [Reuses operator id: 93]
+Output [1]: [d_date_sk#57]
+
+(82) BroadcastHashJoin [codegen id : 27]
+Left keys [1]: [ws_sold_date_sk#45]
+Right keys [1]: [d_date_sk#57]
+Join type: Inner
+Join condition: None
+
+(83) Project [codegen id : 27]
+Output [4]: [ws_quantity#43, ws_list_price#44, c_first_name#55, c_last_name#56]
+Input [6]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_first_name#55, c_last_name#56, d_date_sk#57]
+
+(84) HashAggregate [codegen id : 27]
+Input [4]: [ws_quantity#43, ws_list_price#44, c_first_name#55, c_last_name#56]
+Keys [2]: [c_last_name#56, c_first_name#55]
+Functions [1]: [partial_sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))]
+Aggregate Attributes [2]: [sum#58, isEmpty#59]
+Results [4]: [c_last_name#56, c_first_name#55, sum#60, isEmpty#61]
+
+(85) Exchange
+Input [4]: [c_last_name#56, c_first_name#55, sum#60, isEmpty#61]
+Arguments: hashpartitioning(c_last_name#56, c_first_name#55, 5), ENSURE_REQUIREMENTS, [plan_id=9]
+
+(86) HashAggregate [codegen id : 28]
+Input [4]: [c_last_name#56, c_first_name#55, sum#60, isEmpty#61]
+Keys [2]: [c_last_name#56, c_first_name#55]
+Functions [1]: [sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))]
+Aggregate Attributes [1]: [sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))#62]
+Results [3]: [c_last_name#56, c_first_name#55, sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))#62 AS sales#63]
+
+(87) Union
+
+(88) TakeOrderedAndProject
+Input [3]: [c_last_name#33, c_first_name#32, sales#40]
+Arguments: 100, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#40 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#40]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6
+BroadcastExchange (93)
++- * ColumnarToRow (92)
+ +- CometProject (91)
+ +- CometFilter (90)
+ +- CometScan parquet spark_catalog.default.date_dim (89)
+
+
+(89) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#34, d_year#64, d_moy#65]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(90) CometFilter
+Input [3]: [d_date_sk#34, d_year#64, d_moy#65]
+Condition : ((((isnotnull(d_year#64) AND isnotnull(d_moy#65)) AND (d_year#64 = 2000)) AND (d_moy#65 = 2)) AND isnotnull(d_date_sk#34))
+
+(91) CometProject
+Input [3]: [d_date_sk#34, d_year#64, d_moy#65]
+Arguments: [d_date_sk#34], [d_date_sk#34]
+
+(92) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#34]
+
+(93) BroadcastExchange
+Input [1]: [d_date_sk#34]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10]
+
+Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9
+BroadcastExchange (98)
++- * ColumnarToRow (97)
+ +- CometProject (96)
+ +- CometFilter (95)
+ +- CometScan parquet spark_catalog.default.date_dim (94)
+
+
+(94) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#10, d_date#11, d_year#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(95) CometFilter
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+
+(96) CometProject
+Input [3]: [d_date_sk#10, d_date#11, d_year#12]
+Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11]
+
+(97) ColumnarToRow [codegen id : 1]
+Input [2]: [d_date_sk#10, d_date#11]
+
+(98) BroadcastExchange
+Input [2]: [d_date_sk#10, d_date#11]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11]
+
+Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#29, [id=#30]
+* HashAggregate (116)
++- Exchange (115)
+ +- * HashAggregate (114)
+ +- * HashAggregate (113)
+ +- Exchange (112)
+ +- * ColumnarToRow (111)
+ +- CometHashAggregate (110)
+ +- CometProject (109)
+ +- CometBroadcastHashJoin (108)
+ :- CometProject (103)
+ : +- CometBroadcastHashJoin (102)
+ : :- CometFilter (100)
+ : : +- CometScan parquet spark_catalog.default.store_sales (99)
+ : +- ReusedExchange (101)
+ +- CometBroadcastExchange (107)
+ +- CometProject (106)
+ +- CometFilter (105)
+ +- CometScan parquet spark_catalog.default.date_dim (104)
+
+
+(99) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)]
+PushedFilters: [IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(100) CometFilter
+Input [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69]
+Condition : isnotnull(ss_customer_sk#66)
+
+(101) ReusedExchange [Reuses operator id: 33]
+Output [1]: [c_customer_sk#71]
+
+(102) CometBroadcastHashJoin
+Left output [4]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69]
+Right output [1]: [c_customer_sk#71]
+Arguments: [ss_customer_sk#66], [c_customer_sk#71], Inner, BuildRight
+
+(103) CometProject
+Input [5]: [ss_customer_sk#66, ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71]
+Arguments: [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71], [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71]
+
+(104) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#72, d_year#73]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(105) CometFilter
+Input [2]: [d_date_sk#72, d_year#73]
+Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#72))
+
+(106) CometProject
+Input [2]: [d_date_sk#72, d_year#73]
+Arguments: [d_date_sk#72], [d_date_sk#72]
+
+(107) CometBroadcastExchange
+Input [1]: [d_date_sk#72]
+Arguments: [d_date_sk#72]
+
+(108) CometBroadcastHashJoin
+Left output [4]: [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71]
+Right output [1]: [d_date_sk#72]
+Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight
+
+(109) CometProject
+Input [5]: [ss_quantity#67, ss_sales_price#68, ss_sold_date_sk#69, c_customer_sk#71, d_date_sk#72]
+Arguments: [ss_quantity#67, ss_sales_price#68, c_customer_sk#71], [ss_quantity#67, ss_sales_price#68, c_customer_sk#71]
+
+(110) CometHashAggregate
+Input [3]: [ss_quantity#67, ss_sales_price#68, c_customer_sk#71]
+Keys [1]: [c_customer_sk#71]
+Functions [1]: [partial_sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))]
+
+(111) ColumnarToRow [codegen id : 1]
+Input [3]: [c_customer_sk#71, sum#74, isEmpty#75]
+
+(112) Exchange
+Input [3]: [c_customer_sk#71, sum#74, isEmpty#75]
+Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=12]
+
+(113) HashAggregate [codegen id : 2]
+Input [3]: [c_customer_sk#71, sum#74, isEmpty#75]
+Keys [1]: [c_customer_sk#71]
+Functions [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))#76]
+Results [1]: [sum((cast(ss_quantity#67 as decimal(10,0)) * ss_sales_price#68))#76 AS csales#77]
+
+(114) HashAggregate [codegen id : 2]
+Input [1]: [csales#77]
+Keys: []
+Functions [1]: [partial_max(csales#77)]
+Aggregate Attributes [1]: [max#78]
+Results [1]: [max#79]
+
+(115) Exchange
+Input [1]: [max#79]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13]
+
+(116) HashAggregate [codegen id : 3]
+Input [1]: [max#79]
+Keys: []
+Functions [1]: [max(csales#77)]
+Aggregate Attributes [1]: [max(csales#77)#80]
+Results [1]: [max(csales#77)#80 AS tpcds_cmax#81]
+
+Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#70
+BroadcastExchange (121)
++- * ColumnarToRow (120)
+ +- CometProject (119)
+ +- CometFilter (118)
+ +- CometScan parquet spark_catalog.default.date_dim (117)
+
+
+(117) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#72, d_year#73]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(118) CometFilter
+Input [2]: [d_date_sk#72, d_year#73]
+Condition : (d_year#73 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#72))
+
+(119) CometProject
+Input [2]: [d_date_sk#72, d_year#73]
+Arguments: [d_date_sk#72], [d_date_sk#72]
+
+(120) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#72]
+
+(121) BroadcastExchange
+Input [1]: [d_date_sk#72]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14]
+
+Subquery:5 Hosting operator id = 51 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30]
+
+Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6
+
+Subquery:7 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt
new file mode 100644
index 000000000..54ee3dbde
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt
@@ -0,0 +1,183 @@
+TakeOrderedAndProject [c_last_name,c_first_name,sales]
+ Union
+ WholeStageCodegen (14)
+ HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty]
+ InputAdapter
+ Exchange [c_last_name,c_first_name] #1
+ WholeStageCodegen (13)
+ HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty]
+ Project [cs_quantity,cs_list_price,c_first_name,c_last_name]
+ BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name]
+ BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ SortMergeJoin [cs_bill_customer_sk,c_customer_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [cs_bill_customer_sk]
+ InputAdapter
+ Exchange [cs_bill_customer_sk] #2
+ WholeStageCodegen (3)
+ Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+ BroadcastHashJoin [cs_item_sk,item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [cs_bill_customer_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (2)
+ Project [item_sk]
+ Filter [cnt]
+ HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count]
+ InputAdapter
+ Exchange [_groupingexpression,i_item_sk,d_date] #5
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,d_date]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #6
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk,d_date]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
+ CometBroadcastExchange #7
+ CometProject [d_date_sk,d_date]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
+ CometBroadcastExchange #8
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
+ InputAdapter
+ WholeStageCodegen (6)
+ Sort [c_customer_sk]
+ Project [c_customer_sk]
+ Filter [ssales]
+ Subquery #3
+ WholeStageCodegen (3)
+ HashAggregate [max] [max(csales),tpcds_cmax,max]
+ InputAdapter
+ Exchange #11
+ WholeStageCodegen (2)
+ HashAggregate [csales] [max,max]
+ HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty]
+ InputAdapter
+ Exchange [c_customer_sk] #12
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometProject [ss_quantity,ss_sales_price,c_customer_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometFilter [ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #4
+ BroadcastExchange #13
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ ReusedExchange [c_customer_sk] #10
+ CometBroadcastExchange #14
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
+ InputAdapter
+ Exchange [c_customer_sk] #9
+ WholeStageCodegen (5)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometProject [ss_quantity,ss_sales_price,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
+ CometFilter [ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+ CometBroadcastExchange #10
+ CometFilter [c_customer_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk]
+ InputAdapter
+ BroadcastExchange #15
+ WholeStageCodegen (11)
+ SortMergeJoin [c_customer_sk,c_customer_sk]
+ InputAdapter
+ WholeStageCodegen (8)
+ Sort [c_customer_sk]
+ InputAdapter
+ Exchange [c_customer_sk] #16
+ WholeStageCodegen (7)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [c_customer_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
+ InputAdapter
+ WholeStageCodegen (10)
+ Sort [c_customer_sk]
+ Project [c_customer_sk]
+ Filter [ssales]
+ ReusedSubquery [tpcds_cmax] #3
+ HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
+ InputAdapter
+ ReusedExchange [c_customer_sk,sum,isEmpty] #9
+ InputAdapter
+ ReusedExchange [d_date_sk] #3
+ WholeStageCodegen (28)
+ HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty]
+ InputAdapter
+ Exchange [c_last_name,c_first_name] #17
+ WholeStageCodegen (27)
+ HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty]
+ Project [ws_quantity,ws_list_price,c_first_name,c_last_name]
+ BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name]
+ BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
+ SortMergeJoin [ws_bill_customer_sk,c_customer_sk]
+ InputAdapter
+ WholeStageCodegen (18)
+ Sort [ws_bill_customer_sk]
+ InputAdapter
+ Exchange [ws_bill_customer_sk] #18
+ WholeStageCodegen (17)
+ Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+ BroadcastHashJoin [ws_item_sk,item_sk]
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ws_bill_customer_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ InputAdapter
+ ReusedExchange [item_sk] #4
+ InputAdapter
+ WholeStageCodegen (20)
+ Sort [c_customer_sk]
+ Project [c_customer_sk]
+ Filter [ssales]
+ ReusedSubquery [tpcds_cmax] #3
+ HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
+ InputAdapter
+ ReusedExchange [c_customer_sk,sum,isEmpty] #9
+ InputAdapter
+ ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15
+ InputAdapter
+ ReusedExchange [d_date_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt
new file mode 100644
index 000000000..7241b5ea0
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt
@@ -0,0 +1,427 @@
+== Physical Plan ==
+* Filter (46)
++- * HashAggregate (45)
+ +- Exchange (44)
+ +- * HashAggregate (43)
+ +- * HashAggregate (42)
+ +- Exchange (41)
+ +- * HashAggregate (40)
+ +- * Project (39)
+ +- * BroadcastHashJoin Inner BuildRight (38)
+ :- * Project (33)
+ : +- * BroadcastHashJoin Inner BuildRight (32)
+ : :- * Project (27)
+ : : +- * BroadcastHashJoin Inner BuildRight (26)
+ : : :- * Project (21)
+ : : : +- * BroadcastHashJoin Inner BuildRight (20)
+ : : : :- * Project (14)
+ : : : : +- * SortMergeJoin Inner (13)
+ : : : : :- * Sort (6)
+ : : : : : +- Exchange (5)
+ : : : : : +- * ColumnarToRow (4)
+ : : : : : +- CometProject (3)
+ : : : : : +- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : +- * Sort (12)
+ : : : : +- Exchange (11)
+ : : : : +- * ColumnarToRow (10)
+ : : : : +- CometProject (9)
+ : : : : +- CometFilter (8)
+ : : : : +- CometScan parquet spark_catalog.default.store_returns (7)
+ : : : +- BroadcastExchange (19)
+ : : : +- * ColumnarToRow (18)
+ : : : +- CometProject (17)
+ : : : +- CometFilter (16)
+ : : : +- CometScan parquet spark_catalog.default.store (15)
+ : : +- BroadcastExchange (25)
+ : : +- * ColumnarToRow (24)
+ : : +- CometFilter (23)
+ : : +- CometScan parquet spark_catalog.default.item (22)
+ : +- BroadcastExchange (31)
+ : +- * ColumnarToRow (30)
+ : +- CometFilter (29)
+ : +- CometScan parquet spark_catalog.default.customer (28)
+ +- BroadcastExchange (37)
+ +- * ColumnarToRow (36)
+ +- CometFilter (35)
+ +- CometScan parquet spark_catalog.default.customer_address (34)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6]
+Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2))
+
+(3) CometProject
+Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6]
+Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+
+(4) ColumnarToRow [codegen id : 1]
+Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+
+(5) Exchange
+Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(6) Sort [codegen id : 2]
+Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0
+
+(7) Scan parquet spark_catalog.default.store_returns
+Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_returns]
+PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)]
+ReadSchema: struct
+
+(8) CometFilter
+Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9]
+Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7))
+
+(9) CometProject
+Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9]
+Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8]
+
+(10) ColumnarToRow [codegen id : 3]
+Input [2]: [sr_item_sk#7, sr_ticket_number#8]
+
+(11) Exchange
+Input [2]: [sr_item_sk#7, sr_ticket_number#8]
+Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(12) Sort [codegen id : 4]
+Input [2]: [sr_item_sk#7, sr_ticket_number#8]
+Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0
+
+(13) SortMergeJoin [codegen id : 9]
+Left keys [2]: [ss_ticket_number#4, ss_item_sk#1]
+Right keys [2]: [sr_ticket_number#8, sr_item_sk#7]
+Join type: Inner
+Join condition: None
+
+(14) Project [codegen id : 9]
+Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5]
+Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8]
+
+(15) Scan parquet spark_catalog.default.store
+Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14]
+Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14))
+
+(17) CometProject
+Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14]
+Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+
+(18) ColumnarToRow [codegen id : 5]
+Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+
+(19) BroadcastExchange
+Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+(20) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_store_sk#3]
+Right keys [1]: [s_store_sk#10]
+Join type: Inner
+Join condition: None
+
+(21) Project [codegen id : 9]
+Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14]
+Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+
+(22) Scan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(23) CometFilter
+Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15))
+
+(24) ColumnarToRow [codegen id : 6]
+Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+
+(25) BroadcastExchange
+Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
+
+(26) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_item_sk#1]
+Right keys [1]: [i_item_sk#15]
+Join type: Inner
+Join condition: None
+
+(27) Project [codegen id : 9]
+Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+
+(28) Scan parquet spark_catalog.default.customer
+Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)]
+ReadSchema: struct
+
+(29) CometFilter
+Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24))
+
+(30) ColumnarToRow [codegen id : 7]
+Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+
+(31) BroadcastExchange
+Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+
+(32) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_customer_sk#2]
+Right keys [1]: [c_customer_sk#21]
+Join type: Inner
+Join condition: None
+
+(33) Project [codegen id : 9]
+Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24]
+Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+
+(34) Scan parquet spark_catalog.default.customer_address
+Output [3]: [ca_state#25, ca_zip#26, ca_country#27]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)]
+ReadSchema: struct
+
+(35) CometFilter
+Input [3]: [ca_state#25, ca_zip#26, ca_country#27]
+Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26))
+
+(36) ColumnarToRow [codegen id : 8]
+Input [3]: [ca_state#25, ca_zip#26, ca_country#27]
+
+(37) BroadcastExchange
+Input [3]: [ca_state#25, ca_zip#26, ca_country#27]
+Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6]
+
+(38) BroadcastHashJoin [codegen id : 9]
+Left keys [2]: [c_birth_country#24, s_zip#14]
+Right keys [2]: [upper(ca_country#27), ca_zip#26]
+Join type: Inner
+Join condition: None
+
+(39) Project [codegen id : 9]
+Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25]
+Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27]
+
+(40) HashAggregate [codegen id : 9]
+Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25]
+Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17]
+Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))]
+Aggregate Attributes [1]: [sum#28]
+Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29]
+
+(41) Exchange
+Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29]
+Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+
+(42) HashAggregate [codegen id : 10]
+Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29]
+Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17]
+Functions [1]: [sum(UnscaledValue(ss_net_paid#5))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30]
+Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31]
+
+(43) HashAggregate [codegen id : 10]
+Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31]
+Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11]
+Functions [1]: [partial_sum(netpaid#31)]
+Aggregate Attributes [2]: [sum#32, isEmpty#33]
+Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35]
+
+(44) Exchange
+Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35]
+Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8]
+
+(45) HashAggregate [codegen id : 11]
+Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35]
+Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11]
+Functions [1]: [sum(netpaid#31)]
+Aggregate Attributes [1]: [sum(netpaid#31)#36]
+Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37]
+
+(46) Filter [codegen id : 11]
+Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37]
+Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8))))
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39]
+* HashAggregate (73)
++- Exchange (72)
+ +- * HashAggregate (71)
+ +- * HashAggregate (70)
+ +- Exchange (69)
+ +- * HashAggregate (68)
+ +- * Project (67)
+ +- * BroadcastHashJoin Inner BuildRight (66)
+ :- * Project (64)
+ : +- * BroadcastHashJoin Inner BuildRight (63)
+ : :- * Project (61)
+ : : +- * BroadcastHashJoin Inner BuildRight (60)
+ : : :- * Project (55)
+ : : : +- * BroadcastHashJoin Inner BuildRight (54)
+ : : : :- * Project (52)
+ : : : : +- * SortMergeJoin Inner (51)
+ : : : : :- * Sort (48)
+ : : : : : +- ReusedExchange (47)
+ : : : : +- * Sort (50)
+ : : : : +- ReusedExchange (49)
+ : : : +- ReusedExchange (53)
+ : : +- BroadcastExchange (59)
+ : : +- * ColumnarToRow (58)
+ : : +- CometFilter (57)
+ : : +- CometScan parquet spark_catalog.default.item (56)
+ : +- ReusedExchange (62)
+ +- ReusedExchange (65)
+
+
+(47) ReusedExchange [Reuses operator id: 5]
+Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44]
+
+(48) Sort [codegen id : 2]
+Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44]
+Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0
+
+(49) ReusedExchange [Reuses operator id: 11]
+Output [2]: [sr_item_sk#45, sr_ticket_number#46]
+
+(50) Sort [codegen id : 4]
+Input [2]: [sr_item_sk#45, sr_ticket_number#46]
+Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0
+
+(51) SortMergeJoin [codegen id : 9]
+Left keys [2]: [ss_ticket_number#43, ss_item_sk#40]
+Right keys [2]: [sr_ticket_number#46, sr_item_sk#45]
+Join type: Inner
+Join condition: None
+
+(52) Project [codegen id : 9]
+Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44]
+Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46]
+
+(53) ReusedExchange [Reuses operator id: 19]
+Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50]
+
+(54) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_store_sk#42]
+Right keys [1]: [s_store_sk#47]
+Join type: Inner
+Join condition: None
+
+(55) Project [codegen id : 9]
+Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50]
+Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50]
+
+(56) Scan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(57) CometFilter
+Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Condition : isnotnull(i_item_sk#51)
+
+(58) ColumnarToRow [codegen id : 6]
+Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+
+(59) BroadcastExchange
+Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9]
+
+(60) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_item_sk#40]
+Right keys [1]: [i_item_sk#51]
+Join type: Inner
+Join condition: None
+
+(61) Project [codegen id : 9]
+Output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+
+(62) ReusedExchange [Reuses operator id: 31]
+Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60]
+
+(63) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_customer_sk#41]
+Right keys [1]: [c_customer_sk#57]
+Join type: Inner
+Join condition: None
+
+(64) Project [codegen id : 9]
+Output [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60]
+Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60]
+
+(65) ReusedExchange [Reuses operator id: 37]
+Output [3]: [ca_state#61, ca_zip#62, ca_country#63]
+
+(66) BroadcastHashJoin [codegen id : 9]
+Left keys [2]: [c_birth_country#60, s_zip#50]
+Right keys [2]: [upper(ca_country#63), ca_zip#62]
+Join type: Inner
+Join condition: None
+
+(67) Project [codegen id : 9]
+Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61]
+Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63]
+
+(68) HashAggregate [codegen id : 9]
+Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61]
+Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53]
+Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))]
+Aggregate Attributes [1]: [sum#64]
+Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65]
+
+(69) Exchange
+Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65]
+Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+
+(70) HashAggregate [codegen id : 10]
+Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65]
+Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53]
+Functions [1]: [sum(UnscaledValue(ss_net_paid#44))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30]
+Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66]
+
+(71) HashAggregate [codegen id : 10]
+Input [1]: [netpaid#66]
+Keys: []
+Functions [1]: [partial_avg(netpaid#66)]
+Aggregate Attributes [2]: [sum#67, count#68]
+Results [2]: [sum#69, count#70]
+
+(72) Exchange
+Input [2]: [sum#69, count#70]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11]
+
+(73) HashAggregate [codegen id : 11]
+Input [2]: [sum#69, count#70]
+Keys: []
+Functions [1]: [avg(netpaid#66)]
+Aggregate Attributes [1]: [avg(netpaid#66)#71]
+Results [1]: [(0.05 * avg(netpaid#66)#71) AS (0.05 * avg(netpaid))#72]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt
new file mode 100644
index 000000000..8ebd45fd1
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt
@@ -0,0 +1,118 @@
+WholeStageCodegen (11)
+ Filter [paid]
+ Subquery #1
+ WholeStageCodegen (11)
+ HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count]
+ InputAdapter
+ Exchange #9
+ WholeStageCodegen (10)
+ HashAggregate [netpaid] [sum,count,sum,count]
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum]
+ InputAdapter
+ Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10
+ WholeStageCodegen (9)
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum]
+ Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state]
+ BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip]
+ Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country]
+ BroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid]
+ SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
+ InputAdapter
+ WholeStageCodegen (2)
+ Sort [ss_ticket_number,ss_item_sk]
+ InputAdapter
+ ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [sr_ticket_number,sr_item_sk]
+ InputAdapter
+ ReusedExchange [sr_item_sk,sr_ticket_number] #4
+ InputAdapter
+ ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5
+ InputAdapter
+ BroadcastExchange #11
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
+ InputAdapter
+ ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7
+ InputAdapter
+ ReusedExchange [ca_state,ca_zip,ca_country] #8
+ HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty]
+ InputAdapter
+ Exchange [c_last_name,c_first_name,s_store_name] #1
+ WholeStageCodegen (10)
+ HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty]
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum]
+ InputAdapter
+ Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2
+ WholeStageCodegen (9)
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum]
+ Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state]
+ BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip]
+ Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country]
+ BroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid]
+ SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
+ InputAdapter
+ WholeStageCodegen (2)
+ Sort [ss_ticket_number,ss_item_sk]
+ InputAdapter
+ Exchange [ss_ticket_number,ss_item_sk] #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid]
+ CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [sr_ticket_number,sr_item_sk]
+ InputAdapter
+ Exchange [sr_ticket_number,sr_item_sk] #4
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometProject [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_ticket_number,sr_item_sk]
+ CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (5)
+ ColumnarToRow
+ InputAdapter
+ CometProject [s_store_sk,s_store_name,s_state,s_zip]
+ CometFilter [s_market_id,s_store_sk,s_zip]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_color,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (7)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [c_customer_sk,c_birth_country]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country]
+ InputAdapter
+ BroadcastExchange #8
+ WholeStageCodegen (8)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ca_country,ca_zip]
+ CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt
new file mode 100644
index 000000000..0ac5639b7
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt
@@ -0,0 +1,427 @@
+== Physical Plan ==
+* Filter (46)
++- * HashAggregate (45)
+ +- Exchange (44)
+ +- * HashAggregate (43)
+ +- * HashAggregate (42)
+ +- Exchange (41)
+ +- * HashAggregate (40)
+ +- * Project (39)
+ +- * BroadcastHashJoin Inner BuildRight (38)
+ :- * Project (33)
+ : +- * BroadcastHashJoin Inner BuildRight (32)
+ : :- * Project (27)
+ : : +- * BroadcastHashJoin Inner BuildRight (26)
+ : : :- * Project (21)
+ : : : +- * BroadcastHashJoin Inner BuildRight (20)
+ : : : :- * Project (14)
+ : : : : +- * SortMergeJoin Inner (13)
+ : : : : :- * Sort (6)
+ : : : : : +- Exchange (5)
+ : : : : : +- * ColumnarToRow (4)
+ : : : : : +- CometProject (3)
+ : : : : : +- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : +- * Sort (12)
+ : : : : +- Exchange (11)
+ : : : : +- * ColumnarToRow (10)
+ : : : : +- CometProject (9)
+ : : : : +- CometFilter (8)
+ : : : : +- CometScan parquet spark_catalog.default.store_returns (7)
+ : : : +- BroadcastExchange (19)
+ : : : +- * ColumnarToRow (18)
+ : : : +- CometProject (17)
+ : : : +- CometFilter (16)
+ : : : +- CometScan parquet spark_catalog.default.store (15)
+ : : +- BroadcastExchange (25)
+ : : +- * ColumnarToRow (24)
+ : : +- CometFilter (23)
+ : : +- CometScan parquet spark_catalog.default.item (22)
+ : +- BroadcastExchange (31)
+ : +- * ColumnarToRow (30)
+ : +- CometFilter (29)
+ : +- CometScan parquet spark_catalog.default.customer (28)
+ +- BroadcastExchange (37)
+ +- * ColumnarToRow (36)
+ +- CometFilter (35)
+ +- CometScan parquet spark_catalog.default.customer_address (34)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6]
+Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2))
+
+(3) CometProject
+Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6]
+Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+
+(4) ColumnarToRow [codegen id : 1]
+Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+
+(5) Exchange
+Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(6) Sort [codegen id : 2]
+Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5]
+Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0
+
+(7) Scan parquet spark_catalog.default.store_returns
+Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_returns]
+PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)]
+ReadSchema: struct
+
+(8) CometFilter
+Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9]
+Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7))
+
+(9) CometProject
+Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9]
+Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8]
+
+(10) ColumnarToRow [codegen id : 3]
+Input [2]: [sr_item_sk#7, sr_ticket_number#8]
+
+(11) Exchange
+Input [2]: [sr_item_sk#7, sr_ticket_number#8]
+Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(12) Sort [codegen id : 4]
+Input [2]: [sr_item_sk#7, sr_ticket_number#8]
+Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0
+
+(13) SortMergeJoin [codegen id : 9]
+Left keys [2]: [ss_ticket_number#4, ss_item_sk#1]
+Right keys [2]: [sr_ticket_number#8, sr_item_sk#7]
+Join type: Inner
+Join condition: None
+
+(14) Project [codegen id : 9]
+Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5]
+Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8]
+
+(15) Scan parquet spark_catalog.default.store
+Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14]
+Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14))
+
+(17) CometProject
+Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14]
+Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+
+(18) ColumnarToRow [codegen id : 5]
+Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+
+(19) BroadcastExchange
+Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+(20) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_store_sk#3]
+Right keys [1]: [s_store_sk#10]
+Join type: Inner
+Join condition: None
+
+(21) Project [codegen id : 9]
+Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14]
+Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14]
+
+(22) Scan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(23) CometFilter
+Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon )) AND isnotnull(i_item_sk#15))
+
+(24) ColumnarToRow [codegen id : 6]
+Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+
+(25) BroadcastExchange
+Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
+
+(26) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_item_sk#1]
+Right keys [1]: [i_item_sk#15]
+Join type: Inner
+Join condition: None
+
+(27) Project [codegen id : 9]
+Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20]
+
+(28) Scan parquet spark_catalog.default.customer
+Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)]
+ReadSchema: struct
+
+(29) CometFilter
+Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24))
+
+(30) ColumnarToRow [codegen id : 7]
+Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+
+(31) BroadcastExchange
+Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+
+(32) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_customer_sk#2]
+Right keys [1]: [c_customer_sk#21]
+Join type: Inner
+Join condition: None
+
+(33) Project [codegen id : 9]
+Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24]
+Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24]
+
+(34) Scan parquet spark_catalog.default.customer_address
+Output [3]: [ca_state#25, ca_zip#26, ca_country#27]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)]
+ReadSchema: struct
+
+(35) CometFilter
+Input [3]: [ca_state#25, ca_zip#26, ca_country#27]
+Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26))
+
+(36) ColumnarToRow [codegen id : 8]
+Input [3]: [ca_state#25, ca_zip#26, ca_country#27]
+
+(37) BroadcastExchange
+Input [3]: [ca_state#25, ca_zip#26, ca_country#27]
+Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6]
+
+(38) BroadcastHashJoin [codegen id : 9]
+Left keys [2]: [c_birth_country#24, s_zip#14]
+Right keys [2]: [upper(ca_country#27), ca_zip#26]
+Join type: Inner
+Join condition: None
+
+(39) Project [codegen id : 9]
+Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25]
+Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27]
+
+(40) HashAggregate [codegen id : 9]
+Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25]
+Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17]
+Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))]
+Aggregate Attributes [1]: [sum#28]
+Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29]
+
+(41) Exchange
+Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29]
+Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+
+(42) HashAggregate [codegen id : 10]
+Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29]
+Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17]
+Functions [1]: [sum(UnscaledValue(ss_net_paid#5))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30]
+Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31]
+
+(43) HashAggregate [codegen id : 10]
+Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31]
+Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11]
+Functions [1]: [partial_sum(netpaid#31)]
+Aggregate Attributes [2]: [sum#32, isEmpty#33]
+Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35]
+
+(44) Exchange
+Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35]
+Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8]
+
+(45) HashAggregate [codegen id : 11]
+Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35]
+Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11]
+Functions [1]: [sum(netpaid#31)]
+Aggregate Attributes [1]: [sum(netpaid#31)#36]
+Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37]
+
+(46) Filter [codegen id : 11]
+Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37]
+Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8))))
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39]
+* HashAggregate (73)
++- Exchange (72)
+ +- * HashAggregate (71)
+ +- * HashAggregate (70)
+ +- Exchange (69)
+ +- * HashAggregate (68)
+ +- * Project (67)
+ +- * BroadcastHashJoin Inner BuildRight (66)
+ :- * Project (64)
+ : +- * BroadcastHashJoin Inner BuildRight (63)
+ : :- * Project (61)
+ : : +- * BroadcastHashJoin Inner BuildRight (60)
+ : : :- * Project (55)
+ : : : +- * BroadcastHashJoin Inner BuildRight (54)
+ : : : :- * Project (52)
+ : : : : +- * SortMergeJoin Inner (51)
+ : : : : :- * Sort (48)
+ : : : : : +- ReusedExchange (47)
+ : : : : +- * Sort (50)
+ : : : : +- ReusedExchange (49)
+ : : : +- ReusedExchange (53)
+ : : +- BroadcastExchange (59)
+ : : +- * ColumnarToRow (58)
+ : : +- CometFilter (57)
+ : : +- CometScan parquet spark_catalog.default.item (56)
+ : +- ReusedExchange (62)
+ +- ReusedExchange (65)
+
+
+(47) ReusedExchange [Reuses operator id: 5]
+Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44]
+
+(48) Sort [codegen id : 2]
+Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44]
+Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0
+
+(49) ReusedExchange [Reuses operator id: 11]
+Output [2]: [sr_item_sk#45, sr_ticket_number#46]
+
+(50) Sort [codegen id : 4]
+Input [2]: [sr_item_sk#45, sr_ticket_number#46]
+Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0
+
+(51) SortMergeJoin [codegen id : 9]
+Left keys [2]: [ss_ticket_number#43, ss_item_sk#40]
+Right keys [2]: [sr_ticket_number#46, sr_item_sk#45]
+Join type: Inner
+Join condition: None
+
+(52) Project [codegen id : 9]
+Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44]
+Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46]
+
+(53) ReusedExchange [Reuses operator id: 19]
+Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50]
+
+(54) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_store_sk#42]
+Right keys [1]: [s_store_sk#47]
+Join type: Inner
+Join condition: None
+
+(55) Project [codegen id : 9]
+Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50]
+Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50]
+
+(56) Scan parquet spark_catalog.default.item
+Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(57) CometFilter
+Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Condition : isnotnull(i_item_sk#51)
+
+(58) ColumnarToRow [codegen id : 6]
+Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+
+(59) BroadcastExchange
+Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9]
+
+(60) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_item_sk#40]
+Right keys [1]: [i_item_sk#51]
+Join type: Inner
+Join condition: None
+
+(61) Project [codegen id : 9]
+Output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56]
+
+(62) ReusedExchange [Reuses operator id: 31]
+Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60]
+
+(63) BroadcastHashJoin [codegen id : 9]
+Left keys [1]: [ss_customer_sk#41]
+Right keys [1]: [c_customer_sk#57]
+Join type: Inner
+Join condition: None
+
+(64) Project [codegen id : 9]
+Output [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60]
+Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60]
+
+(65) ReusedExchange [Reuses operator id: 37]
+Output [3]: [ca_state#61, ca_zip#62, ca_country#63]
+
+(66) BroadcastHashJoin [codegen id : 9]
+Left keys [2]: [c_birth_country#60, s_zip#50]
+Right keys [2]: [upper(ca_country#63), ca_zip#62]
+Join type: Inner
+Join condition: None
+
+(67) Project [codegen id : 9]
+Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61]
+Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63]
+
+(68) HashAggregate [codegen id : 9]
+Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61]
+Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53]
+Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))]
+Aggregate Attributes [1]: [sum#64]
+Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65]
+
+(69) Exchange
+Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65]
+Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+
+(70) HashAggregate [codegen id : 10]
+Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65]
+Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53]
+Functions [1]: [sum(UnscaledValue(ss_net_paid#44))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30]
+Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66]
+
+(71) HashAggregate [codegen id : 10]
+Input [1]: [netpaid#66]
+Keys: []
+Functions [1]: [partial_avg(netpaid#66)]
+Aggregate Attributes [2]: [sum#67, count#68]
+Results [2]: [sum#69, count#70]
+
+(72) Exchange
+Input [2]: [sum#69, count#70]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11]
+
+(73) HashAggregate [codegen id : 11]
+Input [2]: [sum#69, count#70]
+Keys: []
+Functions [1]: [avg(netpaid#66)]
+Aggregate Attributes [1]: [avg(netpaid#66)#71]
+Results [1]: [(0.05 * avg(netpaid#66)#71) AS (0.05 * avg(netpaid))#72]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt
new file mode 100644
index 000000000..8ebd45fd1
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt
@@ -0,0 +1,118 @@
+WholeStageCodegen (11)
+ Filter [paid]
+ Subquery #1
+ WholeStageCodegen (11)
+ HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count]
+ InputAdapter
+ Exchange #9
+ WholeStageCodegen (10)
+ HashAggregate [netpaid] [sum,count,sum,count]
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum]
+ InputAdapter
+ Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10
+ WholeStageCodegen (9)
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum]
+ Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state]
+ BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip]
+ Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country]
+ BroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid]
+ SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
+ InputAdapter
+ WholeStageCodegen (2)
+ Sort [ss_ticket_number,ss_item_sk]
+ InputAdapter
+ ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [sr_ticket_number,sr_item_sk]
+ InputAdapter
+ ReusedExchange [sr_item_sk,sr_ticket_number] #4
+ InputAdapter
+ ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5
+ InputAdapter
+ BroadcastExchange #11
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
+ InputAdapter
+ ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7
+ InputAdapter
+ ReusedExchange [ca_state,ca_zip,ca_country] #8
+ HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty]
+ InputAdapter
+ Exchange [c_last_name,c_first_name,s_store_name] #1
+ WholeStageCodegen (10)
+ HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty]
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum]
+ InputAdapter
+ Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2
+ WholeStageCodegen (9)
+ HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum]
+ Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state]
+ BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip]
+ Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country]
+ BroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id]
+ BroadcastHashJoin [ss_item_sk,i_item_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip]
+ BroadcastHashJoin [ss_store_sk,s_store_sk]
+ Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid]
+ SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
+ InputAdapter
+ WholeStageCodegen (2)
+ Sort [ss_ticket_number,ss_item_sk]
+ InputAdapter
+ Exchange [ss_ticket_number,ss_item_sk] #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid]
+ CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk]
+ InputAdapter
+ WholeStageCodegen (4)
+ Sort [sr_ticket_number,sr_item_sk]
+ InputAdapter
+ Exchange [sr_ticket_number,sr_item_sk] #4
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometProject [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_ticket_number,sr_item_sk]
+ CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (5)
+ ColumnarToRow
+ InputAdapter
+ CometProject [s_store_sk,s_store_name,s_state,s_zip]
+ CometFilter [s_market_id,s_store_sk,s_zip]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [i_color,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (7)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [c_customer_sk,c_birth_country]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country]
+ InputAdapter
+ BroadcastExchange #8
+ WholeStageCodegen (8)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ca_country,ca_zip]
+ CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt
new file mode 100644
index 000000000..daff22064
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt
@@ -0,0 +1,311 @@
+== Physical Plan ==
+TakeOrderedAndProject (42)
++- * HashAggregate (41)
+ +- Exchange (40)
+ +- * ColumnarToRow (39)
+ +- CometHashAggregate (38)
+ +- CometProject (37)
+ +- CometBroadcastHashJoin (36)
+ :- CometProject (32)
+ : +- CometBroadcastHashJoin (31)
+ : :- CometProject (27)
+ : : +- CometBroadcastHashJoin (26)
+ : : :- CometProject (24)
+ : : : +- CometBroadcastHashJoin (23)
+ : : : :- CometProject (18)
+ : : : : +- CometBroadcastHashJoin (17)
+ : : : : :- CometProject (12)
+ : : : : : +- CometBroadcastHashJoin (11)
+ : : : : : :- CometProject (7)
+ : : : : : : +- CometBroadcastHashJoin (6)
+ : : : : : : :- CometFilter (2)
+ : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : : : +- CometBroadcastExchange (5)
+ : : : : : : +- CometFilter (4)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3)
+ : : : : : +- CometBroadcastExchange (10)
+ : : : : : +- CometFilter (9)
+ : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8)
+ : : : : +- CometBroadcastExchange (16)
+ : : : : +- CometProject (15)
+ : : : : +- CometFilter (14)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (13)
+ : : : +- CometBroadcastExchange (22)
+ : : : +- CometProject (21)
+ : : : +- CometFilter (20)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (19)
+ : : +- ReusedExchange (25)
+ : +- CometBroadcastExchange (30)
+ : +- CometFilter (29)
+ : +- CometScan parquet spark_catalog.default.store (28)
+ +- CometBroadcastExchange (35)
+ +- CometFilter (34)
+ +- CometScan parquet spark_catalog.default.item (33)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)]
+PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6]
+Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3))
+
+(3) Scan parquet spark_catalog.default.store_returns
+Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)]
+PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12]
+Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10))
+
+(5) CometBroadcastExchange
+Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12]
+Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12]
+
+(6) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6]
+Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12]
+Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight
+
+(7) CometProject
+Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12]
+
+(8) Scan parquet spark_catalog.default.catalog_sales
+Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)]
+PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17]
+Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15))
+
+(10) CometBroadcastExchange
+Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17]
+Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17]
+
+(11) CometBroadcastHashJoin
+Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12]
+Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17]
+Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight
+
+(12) CometProject
+Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17]
+
+(13) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(14) CometFilter
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19))
+
+(15) CometProject
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Arguments: [d_date_sk#19], [d_date_sk#19]
+
+(16) CometBroadcastExchange
+Input [1]: [d_date_sk#19]
+Arguments: [d_date_sk#19]
+
+(17) CometBroadcastHashJoin
+Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#19]
+Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight
+
+(18) CometProject
+Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17]
+
+(19) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(20) CometFilter
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22))
+
+(21) CometProject
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Arguments: [d_date_sk#22], [d_date_sk#22]
+
+(22) CometBroadcastExchange
+Input [1]: [d_date_sk#22]
+Arguments: [d_date_sk#22]
+
+(23) CometBroadcastHashJoin
+Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#22]
+Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight
+
+(24) CometProject
+Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17]
+
+(25) ReusedExchange [Reuses operator id: 22]
+Output [1]: [d_date_sk#25]
+
+(26) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#25]
+Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight
+
+(27) CometProject
+Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16]
+
+(28) Scan parquet spark_catalog.default.store
+Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(29) CometFilter
+Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28]
+Condition : isnotnull(s_store_sk#26)
+
+(30) CometBroadcastExchange
+Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28]
+Arguments: [s_store_sk#26, s_store_id#27, s_store_name#28]
+
+(31) CometBroadcastHashJoin
+Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16]
+Right output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28]
+Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight
+
+(32) CometProject
+Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#27, s_store_name#28]
+Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28]
+
+(33) Scan parquet spark_catalog.default.item
+Output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(34) CometFilter
+Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31]
+Condition : isnotnull(i_item_sk#29)
+
+(35) CometBroadcastExchange
+Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31]
+Arguments: [i_item_sk#29, i_item_id#30, i_item_desc#31]
+
+(36) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28]
+Right output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31]
+Arguments: [ss_item_sk#1], [i_item_sk#29], Inner, BuildRight
+
+(37) CometProject
+Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_sk#29, i_item_id#30, i_item_desc#31]
+Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31]
+
+(38) CometHashAggregate
+Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31]
+Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28]
+Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))]
+
+(39) ColumnarToRow [codegen id : 1]
+Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34]
+
+(40) Exchange
+Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34]
+Arguments: hashpartitioning(i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(41) HashAggregate [codegen id : 2]
+Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34]
+Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28]
+Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))]
+Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#35, sum(UnscaledValue(sr_net_loss#11))#36, sum(UnscaledValue(cs_net_profit#16))#37]
+Results [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#35,17,2) AS store_sales_profit#38, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#36,17,2) AS store_returns_loss#39, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#37,17,2) AS catalog_sales_profit#40]
+
+(42) TakeOrderedAndProject
+Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40]
+Arguments: 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
+BroadcastExchange (47)
++- * ColumnarToRow (46)
+ +- CometProject (45)
+ +- CometFilter (44)
+ +- CometScan parquet spark_catalog.default.date_dim (43)
+
+
+(43) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(44) CometFilter
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19))
+
+(45) CometProject
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Arguments: [d_date_sk#19], [d_date_sk#19]
+
+(46) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#19]
+
+(47) BroadcastExchange
+Input [1]: [d_date_sk#19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13
+BroadcastExchange (52)
++- * ColumnarToRow (51)
+ +- CometProject (50)
+ +- CometFilter (49)
+ +- CometScan parquet spark_catalog.default.date_dim (48)
+
+
+(48) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(49) CometFilter
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22))
+
+(50) CometProject
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Arguments: [d_date_sk#22], [d_date_sk#22]
+
+(51) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#22]
+
+(52) BroadcastExchange
+Input [1]: [d_date_sk#22]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt
new file mode 100644
index 000000000..74a5e8777
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt
@@ -0,0 +1,63 @@
+TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit]
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum]
+ InputAdapter
+ Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit]
+ CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name]
+ CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk]
+ CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk]
+ CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
+ CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #3
+ CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #4
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ CometBroadcastExchange #6
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #7
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ ReusedExchange [d_date_sk] #7
+ CometBroadcastExchange #8
+ CometFilter [s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
+ CometBroadcastExchange #9
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt
new file mode 100644
index 000000000..1369927fe
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt
@@ -0,0 +1,209 @@
+== Physical Plan ==
+TakeOrderedAndProject (30)
++- * HashAggregate (29)
+ +- Exchange (28)
+ +- * ColumnarToRow (27)
+ +- CometHashAggregate (26)
+ +- CometProject (25)
+ +- CometBroadcastHashJoin (24)
+ :- CometProject (19)
+ : +- CometBroadcastHashJoin (18)
+ : :- CometProject (14)
+ : : +- CometBroadcastHashJoin (13)
+ : : :- CometProject (8)
+ : : : +- CometBroadcastHashJoin (7)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : : : +- CometBroadcastExchange (6)
+ : : : +- CometProject (5)
+ : : : +- CometFilter (4)
+ : : : +- CometScan parquet spark_catalog.default.customer_demographics (3)
+ : : +- CometBroadcastExchange (12)
+ : : +- CometProject (11)
+ : : +- CometFilter (10)
+ : : +- CometScan parquet spark_catalog.default.date_dim (9)
+ : +- CometBroadcastExchange (17)
+ : +- CometFilter (16)
+ : +- CometScan parquet spark_catalog.default.item (15)
+ +- CometBroadcastExchange (23)
+ +- CometProject (22)
+ +- CometFilter (21)
+ +- CometScan parquet spark_catalog.default.promotion (20)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)]
+PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8]
+Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3))
+
+(3) Scan parquet spark_catalog.default.customer_demographics
+Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13]
+Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10))
+
+(5) CometProject
+Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13]
+Arguments: [cd_demo_sk#10], [cd_demo_sk#10]
+
+(6) CometBroadcastExchange
+Input [1]: [cd_demo_sk#10]
+Arguments: [cd_demo_sk#10]
+
+(7) CometBroadcastHashJoin
+Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8]
+Right output [1]: [cd_demo_sk#10]
+Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight
+
+(8) CometProject
+Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10]
+Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8]
+
+(9) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#14, d_year#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [2]: [d_date_sk#14, d_year#15]
+Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14))
+
+(11) CometProject
+Input [2]: [d_date_sk#14, d_year#15]
+Arguments: [d_date_sk#14], [d_date_sk#14]
+
+(12) CometBroadcastExchange
+Input [1]: [d_date_sk#14]
+Arguments: [d_date_sk#14]
+
+(13) CometBroadcastHashJoin
+Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8]
+Right output [1]: [d_date_sk#14]
+Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight
+
+(14) CometProject
+Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14]
+Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7]
+
+(15) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#16, i_item_id#17]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [2]: [i_item_sk#16, i_item_id#17]
+Condition : isnotnull(i_item_sk#16)
+
+(17) CometBroadcastExchange
+Input [2]: [i_item_sk#16, i_item_id#17]
+Arguments: [i_item_sk#16, i_item_id#17]
+
+(18) CometBroadcastHashJoin
+Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7]
+Right output [2]: [i_item_sk#16, i_item_id#17]
+Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight
+
+(19) CometProject
+Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#17]
+Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17]
+
+(20) Scan parquet spark_catalog.default.promotion
+Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/promotion]
+PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)]
+ReadSchema: struct
+
+(21) CometFilter
+Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20]
+Condition : (((p_channel_email#19 = N) OR (p_channel_event#20 = N)) AND isnotnull(p_promo_sk#18))
+
+(22) CometProject
+Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20]
+Arguments: [p_promo_sk#18], [p_promo_sk#18]
+
+(23) CometBroadcastExchange
+Input [1]: [p_promo_sk#18]
+Arguments: [p_promo_sk#18]
+
+(24) CometBroadcastHashJoin
+Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17]
+Right output [1]: [p_promo_sk#18]
+Arguments: [cs_promo_sk#3], [p_promo_sk#18], Inner, BuildRight
+
+(25) CometProject
+Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18]
+Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17]
+
+(26) CometHashAggregate
+Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17]
+Keys [1]: [i_item_id#17]
+Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))]
+
+(27) ColumnarToRow [codegen id : 1]
+Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28]
+
+(28) Exchange
+Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28]
+Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(29) HashAggregate [codegen id : 2]
+Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28]
+Keys [1]: [i_item_id#17]
+Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))]
+Aggregate Attributes [4]: [avg(cs_quantity#4)#29, avg(UnscaledValue(cs_list_price#5))#30, avg(UnscaledValue(cs_coupon_amt#7))#31, avg(UnscaledValue(cs_sales_price#6))#32]
+Results [5]: [i_item_id#17, avg(cs_quantity#4)#29 AS agg1#33, cast((avg(UnscaledValue(cs_list_price#5))#30 / 100.0) as decimal(11,6)) AS agg2#34, cast((avg(UnscaledValue(cs_coupon_amt#7))#31 / 100.0) as decimal(11,6)) AS agg3#35, cast((avg(UnscaledValue(cs_sales_price#6))#32 / 100.0) as decimal(11,6)) AS agg4#36]
+
+(30) TakeOrderedAndProject
+Input [5]: [i_item_id#17, agg1#33, agg2#34, agg3#35, agg4#36]
+Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#33, agg2#34, agg3#35, agg4#36]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9
+BroadcastExchange (35)
++- * ColumnarToRow (34)
+ +- CometProject (33)
+ +- CometFilter (32)
+ +- CometScan parquet spark_catalog.default.date_dim (31)
+
+
+(31) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#14, d_year#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(32) CometFilter
+Input [2]: [d_date_sk#14, d_year#15]
+Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14))
+
+(33) CometProject
+Input [2]: [d_date_sk#14, d_year#15]
+Arguments: [d_date_sk#14], [d_date_sk#14]
+
+(34) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#14]
+
+(35) BroadcastExchange
+Input [1]: [d_date_sk#14]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt
new file mode 100644
index 000000000..cba306f68
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt
@@ -0,0 +1,42 @@
+TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count]
+ InputAdapter
+ Exchange [i_item_id] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price]
+ CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id]
+ CometBroadcastHashJoin [cs_promo_sk,p_promo_sk]
+ CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id]
+ CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk]
+ CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #3
+ CometProject [cd_demo_sk]
+ CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
+ CometBroadcastExchange #4
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #5
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
+ CometBroadcastExchange #6
+ CometProject [p_promo_sk]
+ CometFilter [p_channel_email,p_channel_event,p_promo_sk]
+ CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt
new file mode 100644
index 000000000..1692965c6
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt
@@ -0,0 +1,209 @@
+== Physical Plan ==
+TakeOrderedAndProject (30)
++- * HashAggregate (29)
+ +- Exchange (28)
+ +- * ColumnarToRow (27)
+ +- CometHashAggregate (26)
+ +- CometExpand (25)
+ +- CometProject (24)
+ +- CometBroadcastHashJoin (23)
+ :- CometProject (19)
+ : +- CometBroadcastHashJoin (18)
+ : :- CometProject (14)
+ : : +- CometBroadcastHashJoin (13)
+ : : :- CometProject (8)
+ : : : +- CometBroadcastHashJoin (7)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : +- CometBroadcastExchange (6)
+ : : : +- CometProject (5)
+ : : : +- CometFilter (4)
+ : : : +- CometScan parquet spark_catalog.default.customer_demographics (3)
+ : : +- CometBroadcastExchange (12)
+ : : +- CometProject (11)
+ : : +- CometFilter (10)
+ : : +- CometScan parquet spark_catalog.default.date_dim (9)
+ : +- CometBroadcastExchange (17)
+ : +- CometFilter (16)
+ : +- CometScan parquet spark_catalog.default.store (15)
+ +- CometBroadcastExchange (22)
+ +- CometFilter (21)
+ +- CometScan parquet spark_catalog.default.item (20)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)]
+PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8]
+Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1))
+
+(3) Scan parquet spark_catalog.default.customer_demographics
+Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13]
+Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10))
+
+(5) CometProject
+Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13]
+Arguments: [cd_demo_sk#10], [cd_demo_sk#10]
+
+(6) CometBroadcastExchange
+Input [1]: [cd_demo_sk#10]
+Arguments: [cd_demo_sk#10]
+
+(7) CometBroadcastHashJoin
+Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8]
+Right output [1]: [cd_demo_sk#10]
+Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight
+
+(8) CometProject
+Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8]
+
+(9) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#14, d_year#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [2]: [d_date_sk#14, d_year#15]
+Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14))
+
+(11) CometProject
+Input [2]: [d_date_sk#14, d_year#15]
+Arguments: [d_date_sk#14], [d_date_sk#14]
+
+(12) CometBroadcastExchange
+Input [1]: [d_date_sk#14]
+Arguments: [d_date_sk#14]
+
+(13) CometBroadcastHashJoin
+Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8]
+Right output [1]: [d_date_sk#14]
+Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight
+
+(14) CometProject
+Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7]
+
+(15) Scan parquet spark_catalog.default.store
+Output [2]: [s_store_sk#16, s_state#17]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [2]: [s_store_sk#16, s_state#17]
+Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16))
+
+(17) CometBroadcastExchange
+Input [2]: [s_store_sk#16, s_state#17]
+Arguments: [s_store_sk#16, s_state#17]
+
+(18) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7]
+Right output [2]: [s_store_sk#16, s_state#17]
+Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight
+
+(19) CometProject
+Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#17]
+Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17]
+
+(20) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#18, i_item_id#19]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(21) CometFilter
+Input [2]: [i_item_sk#18, i_item_id#19]
+Condition : isnotnull(i_item_sk#18)
+
+(22) CometBroadcastExchange
+Input [2]: [i_item_sk#18, i_item_id#19]
+Arguments: [i_item_sk#18, i_item_id#19]
+
+(23) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17]
+Right output [2]: [i_item_sk#18, i_item_id#19]
+Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight
+
+(24) CometProject
+Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#19]
+Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17]
+
+(25) CometExpand
+Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17]
+Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22]
+
+(26) CometHashAggregate
+Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22]
+Keys [3]: [i_item_id#20, s_state#21, spark_grouping_id#22]
+Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))]
+
+(27) ColumnarToRow [codegen id : 1]
+Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30]
+
+(28) Exchange
+Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30]
+Arguments: hashpartitioning(i_item_id#20, s_state#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(29) HashAggregate [codegen id : 2]
+Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30]
+Keys [3]: [i_item_id#20, s_state#21, spark_grouping_id#22]
+Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))]
+Aggregate Attributes [4]: [avg(ss_quantity#4)#31, avg(UnscaledValue(ss_list_price#5))#32, avg(UnscaledValue(ss_coupon_amt#7))#33, avg(UnscaledValue(ss_sales_price#6))#34]
+Results [7]: [i_item_id#20, s_state#21, cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) AS g_state#35, avg(ss_quantity#4)#31 AS agg1#36, cast((avg(UnscaledValue(ss_list_price#5))#32 / 100.0) as decimal(11,6)) AS agg2#37, cast((avg(UnscaledValue(ss_coupon_amt#7))#33 / 100.0) as decimal(11,6)) AS agg3#38, cast((avg(UnscaledValue(ss_sales_price#6))#34 / 100.0) as decimal(11,6)) AS agg4#39]
+
+(30) TakeOrderedAndProject
+Input [7]: [i_item_id#20, s_state#21, g_state#35, agg1#36, agg2#37, agg3#38, agg4#39]
+Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_item_id#20, s_state#21, g_state#35, agg1#36, agg2#37, agg3#38, agg4#39]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9
+BroadcastExchange (35)
++- * ColumnarToRow (34)
+ +- CometProject (33)
+ +- CometFilter (32)
+ +- CometScan parquet spark_catalog.default.date_dim (31)
+
+
+(31) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#14, d_year#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(32) CometFilter
+Input [2]: [d_date_sk#14, d_year#15]
+Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14))
+
+(33) CometProject
+Input [2]: [d_date_sk#14, d_year#15]
+Arguments: [d_date_sk#14], [d_date_sk#14]
+
+(34) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#14]
+
+(35) BroadcastExchange
+Input [1]: [d_date_sk#14]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt
new file mode 100644
index 000000000..7fcbe967a
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt
@@ -0,0 +1,42 @@
+TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count]
+ InputAdapter
+ Exchange [i_item_id,s_state,spark_grouping_id] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price]
+ CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id]
+ CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state]
+ CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
+ CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk]
+ CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #3
+ CometProject [cd_demo_sk]
+ CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
+ CometBroadcastExchange #4
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #5
+ CometFilter [s_state,s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
+ CometBroadcastExchange #6
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt
new file mode 100644
index 000000000..92bde6faa
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt
@@ -0,0 +1,425 @@
+== Physical Plan ==
+* BroadcastNestedLoopJoin Inner BuildRight (70)
+:- * BroadcastNestedLoopJoin Inner BuildRight (58)
+: :- * BroadcastNestedLoopJoin Inner BuildRight (46)
+: : :- * BroadcastNestedLoopJoin Inner BuildRight (34)
+: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22)
+: : : : :- * HashAggregate (10)
+: : : : : +- Exchange (9)
+: : : : : +- * HashAggregate (8)
+: : : : : +- * HashAggregate (7)
+: : : : : +- Exchange (6)
+: : : : : +- * ColumnarToRow (5)
+: : : : : +- CometHashAggregate (4)
+: : : : : +- CometProject (3)
+: : : : : +- CometFilter (2)
+: : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+: : : : +- BroadcastExchange (21)
+: : : : +- * HashAggregate (20)
+: : : : +- Exchange (19)
+: : : : +- * HashAggregate (18)
+: : : : +- * HashAggregate (17)
+: : : : +- Exchange (16)
+: : : : +- * ColumnarToRow (15)
+: : : : +- CometHashAggregate (14)
+: : : : +- CometProject (13)
+: : : : +- CometFilter (12)
+: : : : +- CometScan parquet spark_catalog.default.store_sales (11)
+: : : +- BroadcastExchange (33)
+: : : +- * HashAggregate (32)
+: : : +- Exchange (31)
+: : : +- * HashAggregate (30)
+: : : +- * HashAggregate (29)
+: : : +- Exchange (28)
+: : : +- * ColumnarToRow (27)
+: : : +- CometHashAggregate (26)
+: : : +- CometProject (25)
+: : : +- CometFilter (24)
+: : : +- CometScan parquet spark_catalog.default.store_sales (23)
+: : +- BroadcastExchange (45)
+: : +- * HashAggregate (44)
+: : +- Exchange (43)
+: : +- * HashAggregate (42)
+: : +- * HashAggregate (41)
+: : +- Exchange (40)
+: : +- * ColumnarToRow (39)
+: : +- CometHashAggregate (38)
+: : +- CometProject (37)
+: : +- CometFilter (36)
+: : +- CometScan parquet spark_catalog.default.store_sales (35)
+: +- BroadcastExchange (57)
+: +- * HashAggregate (56)
+: +- Exchange (55)
+: +- * HashAggregate (54)
+: +- * HashAggregate (53)
+: +- Exchange (52)
+: +- * ColumnarToRow (51)
+: +- CometHashAggregate (50)
+: +- CometProject (49)
+: +- CometFilter (48)
+: +- CometScan parquet spark_catalog.default.store_sales (47)
++- BroadcastExchange (69)
+ +- * HashAggregate (68)
+ +- Exchange (67)
+ +- * HashAggregate (66)
+ +- * HashAggregate (65)
+ +- Exchange (64)
+ +- * ColumnarToRow (63)
+ +- CometHashAggregate (62)
+ +- CometProject (61)
+ +- CometFilter (60)
+ +- CometScan parquet spark_catalog.default.store_sales (59)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))]
+ReadSchema: struct
+
+(2) CometFilter
+Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
+Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00))))
+
+(3) CometProject
+Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5]
+Arguments: [ss_list_price#3], [ss_list_price#3]
+
+(4) CometHashAggregate
+Input [1]: [ss_list_price#3]
+Keys [1]: [ss_list_price#3]
+Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)]
+
+(5) ColumnarToRow [codegen id : 1]
+Input [4]: [ss_list_price#3, sum#6, count#7, count#8]
+
+(6) Exchange
+Input [4]: [ss_list_price#3, sum#6, count#7, count#8]
+Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(7) HashAggregate [codegen id : 2]
+Input [4]: [ss_list_price#3, sum#6, count#7, count#8]
+Keys [1]: [ss_list_price#3]
+Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)]
+Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10]
+Results [4]: [ss_list_price#3, sum#6, count#7, count#8]
+
+(8) HashAggregate [codegen id : 2]
+Input [4]: [ss_list_price#3, sum#6, count#7, count#8]
+Keys: []
+Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11]
+Results [4]: [sum#6, count#7, count#8, count#12]
+
+(9) Exchange
+Input [4]: [sum#6, count#7, count#8, count#12]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2]
+
+(10) HashAggregate [codegen id : 18]
+Input [4]: [sum#6, count#7, count#8, count#12]
+Keys: []
+Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11]
+Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#9 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#10 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15]
+
+(11) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))]
+ReadSchema: struct
+
+(12) CometFilter
+Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20]
+Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00))))
+
+(13) CometProject
+Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20]
+Arguments: [ss_list_price#18], [ss_list_price#18]
+
+(14) CometHashAggregate
+Input [1]: [ss_list_price#18]
+Keys [1]: [ss_list_price#18]
+Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)]
+
+(15) ColumnarToRow [codegen id : 3]
+Input [4]: [ss_list_price#18, sum#21, count#22, count#23]
+
+(16) Exchange
+Input [4]: [ss_list_price#18, sum#21, count#22, count#23]
+Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(17) HashAggregate [codegen id : 4]
+Input [4]: [ss_list_price#18, sum#21, count#22, count#23]
+Keys [1]: [ss_list_price#18]
+Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)]
+Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25]
+Results [4]: [ss_list_price#18, sum#21, count#22, count#23]
+
+(18) HashAggregate [codegen id : 4]
+Input [4]: [ss_list_price#18, sum#21, count#22, count#23]
+Keys: []
+Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26]
+Results [4]: [sum#21, count#22, count#23, count#27]
+
+(19) Exchange
+Input [4]: [sum#21, count#22, count#23, count#27]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4]
+
+(20) HashAggregate [codegen id : 5]
+Input [4]: [sum#21, count#22, count#23, count#27]
+Keys: []
+Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26]
+Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#24 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#25 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30]
+
+(21) BroadcastExchange
+Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30]
+Arguments: IdentityBroadcastMode, [plan_id=5]
+
+(22) BroadcastNestedLoopJoin [codegen id : 18]
+Join type: Inner
+Join condition: None
+
+(23) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))]
+ReadSchema: struct
+
+(24) CometFilter
+Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35]
+Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00))))
+
+(25) CometProject
+Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35]
+Arguments: [ss_list_price#33], [ss_list_price#33]
+
+(26) CometHashAggregate
+Input [1]: [ss_list_price#33]
+Keys [1]: [ss_list_price#33]
+Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)]
+
+(27) ColumnarToRow [codegen id : 6]
+Input [4]: [ss_list_price#33, sum#36, count#37, count#38]
+
+(28) Exchange
+Input [4]: [ss_list_price#33, sum#36, count#37, count#38]
+Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+
+(29) HashAggregate [codegen id : 7]
+Input [4]: [ss_list_price#33, sum#36, count#37, count#38]
+Keys [1]: [ss_list_price#33]
+Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)]
+Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40]
+Results [4]: [ss_list_price#33, sum#36, count#37, count#38]
+
+(30) HashAggregate [codegen id : 7]
+Input [4]: [ss_list_price#33, sum#36, count#37, count#38]
+Keys: []
+Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41]
+Results [4]: [sum#36, count#37, count#38, count#42]
+
+(31) Exchange
+Input [4]: [sum#36, count#37, count#38, count#42]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7]
+
+(32) HashAggregate [codegen id : 8]
+Input [4]: [sum#36, count#37, count#38, count#42]
+Keys: []
+Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41]
+Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#39 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#40 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45]
+
+(33) BroadcastExchange
+Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45]
+Arguments: IdentityBroadcastMode, [plan_id=8]
+
+(34) BroadcastNestedLoopJoin [codegen id : 18]
+Join type: Inner
+Join condition: None
+
+(35) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))]
+ReadSchema: struct
+
+(36) CometFilter
+Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50]
+Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00))))
+
+(37) CometProject
+Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50]
+Arguments: [ss_list_price#48], [ss_list_price#48]
+
+(38) CometHashAggregate
+Input [1]: [ss_list_price#48]
+Keys [1]: [ss_list_price#48]
+Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)]
+
+(39) ColumnarToRow [codegen id : 9]
+Input [4]: [ss_list_price#48, sum#51, count#52, count#53]
+
+(40) Exchange
+Input [4]: [ss_list_price#48, sum#51, count#52, count#53]
+Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9]
+
+(41) HashAggregate [codegen id : 10]
+Input [4]: [ss_list_price#48, sum#51, count#52, count#53]
+Keys [1]: [ss_list_price#48]
+Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)]
+Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55]
+Results [4]: [ss_list_price#48, sum#51, count#52, count#53]
+
+(42) HashAggregate [codegen id : 10]
+Input [4]: [ss_list_price#48, sum#51, count#52, count#53]
+Keys: []
+Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56]
+Results [4]: [sum#51, count#52, count#53, count#57]
+
+(43) Exchange
+Input [4]: [sum#51, count#52, count#53, count#57]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10]
+
+(44) HashAggregate [codegen id : 11]
+Input [4]: [sum#51, count#52, count#53, count#57]
+Keys: []
+Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56]
+Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#54 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#55 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60]
+
+(45) BroadcastExchange
+Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60]
+Arguments: IdentityBroadcastMode, [plan_id=11]
+
+(46) BroadcastNestedLoopJoin [codegen id : 18]
+Join type: Inner
+Join condition: None
+
+(47) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))]
+ReadSchema: struct
+
+(48) CometFilter
+Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65]
+Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00))))
+
+(49) CometProject
+Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65]
+Arguments: [ss_list_price#63], [ss_list_price#63]
+
+(50) CometHashAggregate
+Input [1]: [ss_list_price#63]
+Keys [1]: [ss_list_price#63]
+Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)]
+
+(51) ColumnarToRow [codegen id : 12]
+Input [4]: [ss_list_price#63, sum#66, count#67, count#68]
+
+(52) Exchange
+Input [4]: [ss_list_price#63, sum#66, count#67, count#68]
+Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12]
+
+(53) HashAggregate [codegen id : 13]
+Input [4]: [ss_list_price#63, sum#66, count#67, count#68]
+Keys [1]: [ss_list_price#63]
+Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)]
+Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70]
+Results [4]: [ss_list_price#63, sum#66, count#67, count#68]
+
+(54) HashAggregate [codegen id : 13]
+Input [4]: [ss_list_price#63, sum#66, count#67, count#68]
+Keys: []
+Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71]
+Results [4]: [sum#66, count#67, count#68, count#72]
+
+(55) Exchange
+Input [4]: [sum#66, count#67, count#68, count#72]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13]
+
+(56) HashAggregate [codegen id : 14]
+Input [4]: [sum#66, count#67, count#68, count#72]
+Keys: []
+Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71]
+Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#69 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#70 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75]
+
+(57) BroadcastExchange
+Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75]
+Arguments: IdentityBroadcastMode, [plan_id=14]
+
+(58) BroadcastNestedLoopJoin [codegen id : 18]
+Join type: Inner
+Join condition: None
+
+(59) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store_sales]
+PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))]
+ReadSchema: struct
+
+(60) CometFilter
+Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80]
+Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00))))
+
+(61) CometProject
+Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80]
+Arguments: [ss_list_price#78], [ss_list_price#78]
+
+(62) CometHashAggregate
+Input [1]: [ss_list_price#78]
+Keys [1]: [ss_list_price#78]
+Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)]
+
+(63) ColumnarToRow [codegen id : 15]
+Input [4]: [ss_list_price#78, sum#81, count#82, count#83]
+
+(64) Exchange
+Input [4]: [ss_list_price#78, sum#81, count#82, count#83]
+Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15]
+
+(65) HashAggregate [codegen id : 16]
+Input [4]: [ss_list_price#78, sum#81, count#82, count#83]
+Keys [1]: [ss_list_price#78]
+Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)]
+Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85]
+Results [4]: [ss_list_price#78, sum#81, count#82, count#83]
+
+(66) HashAggregate [codegen id : 16]
+Input [4]: [ss_list_price#78, sum#81, count#82, count#83]
+Keys: []
+Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86]
+Results [4]: [sum#81, count#82, count#83, count#87]
+
+(67) Exchange
+Input [4]: [sum#81, count#82, count#83, count#87]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16]
+
+(68) HashAggregate [codegen id : 17]
+Input [4]: [sum#81, count#82, count#83, count#87]
+Keys: []
+Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)]
+Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86]
+Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#84 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#85 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90]
+
+(69) BroadcastExchange
+Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90]
+Arguments: IdentityBroadcastMode, [plan_id=17]
+
+(70) BroadcastNestedLoopJoin [codegen id : 18]
+Join type: Inner
+Join condition: None
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt
new file mode 100644
index 000000000..47a4fe2c7
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt
@@ -0,0 +1,111 @@
+WholeStageCodegen (18)
+ BroadcastNestedLoopJoin
+ BroadcastNestedLoopJoin
+ BroadcastNestedLoopJoin
+ BroadcastNestedLoopJoin
+ BroadcastNestedLoopJoin
+ HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count]
+ InputAdapter
+ Exchange #1
+ WholeStageCodegen (2)
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
+ InputAdapter
+ Exchange [ss_list_price] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_list_price]
+ CometProject [ss_list_price]
+ CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ InputAdapter
+ BroadcastExchange #3
+ WholeStageCodegen (5)
+ HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count]
+ InputAdapter
+ Exchange #4
+ WholeStageCodegen (4)
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
+ InputAdapter
+ Exchange [ss_list_price] #5
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_list_price]
+ CometProject [ss_list_price]
+ CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (8)
+ HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count]
+ InputAdapter
+ Exchange #7
+ WholeStageCodegen (7)
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
+ InputAdapter
+ Exchange [ss_list_price] #8
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_list_price]
+ CometProject [ss_list_price]
+ CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ InputAdapter
+ BroadcastExchange #9
+ WholeStageCodegen (11)
+ HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count]
+ InputAdapter
+ Exchange #10
+ WholeStageCodegen (10)
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
+ InputAdapter
+ Exchange [ss_list_price] #11
+ WholeStageCodegen (9)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_list_price]
+ CometProject [ss_list_price]
+ CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ InputAdapter
+ BroadcastExchange #12
+ WholeStageCodegen (14)
+ HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count]
+ InputAdapter
+ Exchange #13
+ WholeStageCodegen (13)
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
+ InputAdapter
+ Exchange [ss_list_price] #14
+ WholeStageCodegen (12)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_list_price]
+ CometProject [ss_list_price]
+ CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
+ InputAdapter
+ BroadcastExchange #15
+ WholeStageCodegen (17)
+ HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count]
+ InputAdapter
+ Exchange #16
+ WholeStageCodegen (16)
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count]
+ HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count]
+ InputAdapter
+ Exchange [ss_list_price] #17
+ WholeStageCodegen (15)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_list_price]
+ CometProject [ss_list_price]
+ CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt
new file mode 100644
index 000000000..3c350ad07
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt
@@ -0,0 +1,358 @@
+== Physical Plan ==
+TakeOrderedAndProject (45)
++- * HashAggregate (44)
+ +- Exchange (43)
+ +- * ColumnarToRow (42)
+ +- CometHashAggregate (41)
+ +- CometProject (40)
+ +- CometBroadcastHashJoin (39)
+ :- CometProject (35)
+ : +- CometBroadcastHashJoin (34)
+ : :- CometProject (30)
+ : : +- CometBroadcastHashJoin (29)
+ : : :- CometProject (24)
+ : : : +- CometBroadcastHashJoin (23)
+ : : : :- CometProject (18)
+ : : : : +- CometBroadcastHashJoin (17)
+ : : : : :- CometProject (12)
+ : : : : : +- CometBroadcastHashJoin (11)
+ : : : : : :- CometProject (7)
+ : : : : : : +- CometBroadcastHashJoin (6)
+ : : : : : : :- CometFilter (2)
+ : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : : : +- CometBroadcastExchange (5)
+ : : : : : : +- CometFilter (4)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3)
+ : : : : : +- CometBroadcastExchange (10)
+ : : : : : +- CometFilter (9)
+ : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8)
+ : : : : +- CometBroadcastExchange (16)
+ : : : : +- CometProject (15)
+ : : : : +- CometFilter (14)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (13)
+ : : : +- CometBroadcastExchange (22)
+ : : : +- CometProject (21)
+ : : : +- CometFilter (20)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (19)
+ : : +- CometBroadcastExchange (28)
+ : : +- CometProject (27)
+ : : +- CometFilter (26)
+ : : +- CometScan parquet spark_catalog.default.date_dim (25)
+ : +- CometBroadcastExchange (33)
+ : +- CometFilter (32)
+ : +- CometScan parquet spark_catalog.default.store (31)
+ +- CometBroadcastExchange (38)
+ +- CometFilter (37)
+ +- CometScan parquet spark_catalog.default.item (36)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)]
+PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
+Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3))
+
+(3) Scan parquet spark_catalog.default.store_returns
+Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)]
+PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10))
+
+(5) CometBroadcastExchange
+Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+
+(6) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6]
+Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight
+
+(7) CometProject
+Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12]
+
+(8) Scan parquet spark_catalog.default.catalog_sales
+Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)]
+PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15))
+
+(10) CometBroadcastExchange
+Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+
+(11) CometBroadcastHashJoin
+Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12]
+Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight
+
+(12) CometProject
+Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+
+(13) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(14) CometFilter
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19))
+
+(15) CometProject
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Arguments: [d_date_sk#19], [d_date_sk#19]
+
+(16) CometBroadcastExchange
+Input [1]: [d_date_sk#19]
+Arguments: [d_date_sk#19]
+
+(17) CometBroadcastHashJoin
+Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#19]
+Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight
+
+(18) CometProject
+Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+
+(19) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(20) CometFilter
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22))
+
+(21) CometProject
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Arguments: [d_date_sk#22], [d_date_sk#22]
+
+(22) CometBroadcastExchange
+Input [1]: [d_date_sk#22]
+Arguments: [d_date_sk#22]
+
+(23) CometBroadcastHashJoin
+Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#22]
+Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight
+
+(24) CometProject
+Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17]
+
+(25) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#25, d_year#26]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(26) CometFilter
+Input [2]: [d_date_sk#25, d_year#26]
+Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25))
+
+(27) CometProject
+Input [2]: [d_date_sk#25, d_year#26]
+Arguments: [d_date_sk#25], [d_date_sk#25]
+
+(28) CometBroadcastExchange
+Input [1]: [d_date_sk#25]
+Arguments: [d_date_sk#25]
+
+(29) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#25]
+Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight
+
+(30) CometProject
+Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25]
+Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16]
+
+(31) Scan parquet spark_catalog.default.store
+Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(32) CometFilter
+Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29]
+Condition : isnotnull(s_store_sk#27)
+
+(33) CometBroadcastExchange
+Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29]
+Arguments: [s_store_sk#27, s_store_id#28, s_store_name#29]
+
+(34) CometBroadcastHashJoin
+Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16]
+Right output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29]
+Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight
+
+(35) CometProject
+Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#28, s_store_name#29]
+Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29]
+
+(36) Scan parquet spark_catalog.default.item
+Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(37) CometFilter
+Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32]
+Condition : isnotnull(i_item_sk#30)
+
+(38) CometBroadcastExchange
+Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32]
+Arguments: [i_item_sk#30, i_item_id#31, i_item_desc#32]
+
+(39) CometBroadcastHashJoin
+Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29]
+Right output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32]
+Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight
+
+(40) CometProject
+Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_sk#30, i_item_id#31, i_item_desc#32]
+Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32]
+
+(41) CometHashAggregate
+Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32]
+Keys [4]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29]
+Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)]
+
+(42) ColumnarToRow [codegen id : 1]
+Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#33, sum#34, sum#35]
+
+(43) Exchange
+Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#33, sum#34, sum#35]
+Arguments: hashpartitioning(i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(44) HashAggregate [codegen id : 2]
+Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#33, sum#34, sum#35]
+Keys [4]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29]
+Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)]
+Aggregate Attributes [3]: [sum(ss_quantity#5)#36, sum(sr_return_quantity#11)#37, sum(cs_quantity#16)#38]
+Results [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum(ss_quantity#5)#36 AS store_sales_quantity#39, sum(sr_return_quantity#11)#37 AS store_returns_quantity#40, sum(cs_quantity#16)#38 AS catalog_sales_quantity#41]
+
+(45) TakeOrderedAndProject
+Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41]
+Arguments: 100, [i_item_id#31 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
+BroadcastExchange (50)
++- * ColumnarToRow (49)
+ +- CometProject (48)
+ +- CometFilter (47)
+ +- CometScan parquet spark_catalog.default.date_dim (46)
+
+
+(46) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(47) CometFilter
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19))
+
+(48) CometProject
+Input [3]: [d_date_sk#19, d_year#20, d_moy#21]
+Arguments: [d_date_sk#19], [d_date_sk#19]
+
+(49) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#19]
+
+(50) BroadcastExchange
+Input [1]: [d_date_sk#19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13
+BroadcastExchange (55)
++- * ColumnarToRow (54)
+ +- CometProject (53)
+ +- CometFilter (52)
+ +- CometScan parquet spark_catalog.default.date_dim (51)
+
+
+(51) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(52) CometFilter
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22))
+
+(53) CometProject
+Input [3]: [d_date_sk#22, d_year#23, d_moy#24]
+Arguments: [d_date_sk#22], [d_date_sk#22]
+
+(54) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#22]
+
+(55) BroadcastExchange
+Input [1]: [d_date_sk#22]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
+
+Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18
+BroadcastExchange (60)
++- * ColumnarToRow (59)
+ +- CometProject (58)
+ +- CometFilter (57)
+ +- CometScan parquet spark_catalog.default.date_dim (56)
+
+
+(56) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#25, d_year#26]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(57) CometFilter
+Input [2]: [d_date_sk#25, d_year#26]
+Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25))
+
+(58) CometProject
+Input [2]: [d_date_sk#25, d_year#26]
+Arguments: [d_date_sk#25], [d_date_sk#25]
+
+(59) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#25]
+
+(60) BroadcastExchange
+Input [1]: [d_date_sk#25]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt
new file mode 100644
index 000000000..2bcef6168
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt
@@ -0,0 +1,73 @@
+TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity]
+ WholeStageCodegen (2)
+ HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum]
+ InputAdapter
+ Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity]
+ CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name]
+ CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk]
+ CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk]
+ CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk]
+ CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number]
+ CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #3
+ CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number]
+ CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #4
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #3
+ BroadcastExchange #6
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #7
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #8
+ CometProject [d_date_sk]
+ CometFilter [d_moy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #9
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #10
+ CometFilter [s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
+ CometBroadcastExchange #11
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt
new file mode 100644
index 000000000..98092ef29
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt
@@ -0,0 +1,113 @@
+== Physical Plan ==
+TakeOrderedAndProject (19)
++- * HashAggregate (18)
+ +- Exchange (17)
+ +- * ColumnarToRow (16)
+ +- CometHashAggregate (15)
+ +- CometProject (14)
+ +- CometBroadcastHashJoin (13)
+ :- CometProject (8)
+ : +- CometBroadcastHashJoin (7)
+ : :- CometProject (3)
+ : : +- CometFilter (2)
+ : : +- CometScan parquet spark_catalog.default.date_dim (1)
+ : +- CometBroadcastExchange (6)
+ : +- CometFilter (5)
+ : +- CometScan parquet spark_catalog.default.store_sales (4)
+ +- CometBroadcastExchange (12)
+ +- CometProject (11)
+ +- CometFilter (10)
+ +- CometScan parquet spark_catalog.default.item (9)
+
+
+(1) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#1, d_year#2, d_moy#3]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [d_date_sk#1, d_year#2, d_moy#3]
+Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1))
+
+(3) CometProject
+Input [3]: [d_date_sk#1, d_year#2, d_moy#3]
+Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2]
+
+(4) Scan parquet spark_catalog.default.store_sales
+Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)]
+PushedFilters: [IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(5) CometFilter
+Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6]
+Condition : isnotnull(ss_item_sk#4)
+
+(6) CometBroadcastExchange
+Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6]
+Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6]
+
+(7) CometBroadcastHashJoin
+Left output [2]: [d_date_sk#1, d_year#2]
+Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6]
+Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight
+
+(8) CometProject
+Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6]
+Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5]
+
+(9) Scan parquet spark_catalog.default.item
+Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10]
+Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7))
+
+(11) CometProject
+Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10]
+Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9]
+
+(12) CometBroadcastExchange
+Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9]
+Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9]
+
+(13) CometBroadcastHashJoin
+Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5]
+Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#9]
+Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight
+
+(14) CometProject
+Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9]
+Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9]
+
+(15) CometHashAggregate
+Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9]
+Keys [3]: [d_year#2, i_brand#9, i_brand_id#8]
+Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))]
+
+(16) ColumnarToRow [codegen id : 1]
+Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11]
+
+(17) Exchange
+Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11]
+Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(18) HashAggregate [codegen id : 2]
+Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11]
+Keys [3]: [d_year#2, i_brand#9, i_brand_id#8]
+Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#12]
+Results [4]: [d_year#2, i_brand_id#8 AS brand_id#13, i_brand#9 AS brand#14, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#12,17,2) AS sum_agg#15]
+
+(19) TakeOrderedAndProject
+Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15]
+Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15]
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt
new file mode 100644
index 000000000..cc9c4edf3
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt
@@ -0,0 +1,23 @@
+TakeOrderedAndProject [d_year,sum_agg,brand_id,brand]
+ WholeStageCodegen (2)
+ HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum]
+ InputAdapter
+ Exchange [d_year,i_brand,i_brand_id] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price]
+ CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [d_year,ss_item_sk,ss_ext_sales_price]
+ CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometProject [d_date_sk,d_year]
+ CometFilter [d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #2
+ CometFilter [ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
+ CometBroadcastExchange #3
+ CometProject [i_item_sk,i_brand_id,i_brand]
+ CometFilter [i_manufact_id,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt
new file mode 100644
index 000000000..f4a0a6d51
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt
@@ -0,0 +1,331 @@
+== Physical Plan ==
+TakeOrderedAndProject (51)
++- * Project (50)
+ +- * BroadcastHashJoin Inner BuildRight (49)
+ :- * Project (43)
+ : +- * BroadcastHashJoin Inner BuildRight (42)
+ : :- * Project (37)
+ : : +- * BroadcastHashJoin Inner BuildRight (36)
+ : : :- * Filter (18)
+ : : : +- * HashAggregate (17)
+ : : : +- Exchange (16)
+ : : : +- * ColumnarToRow (15)
+ : : : +- CometHashAggregate (14)
+ : : : +- CometProject (13)
+ : : : +- CometBroadcastHashJoin (12)
+ : : : :- CometProject (8)
+ : : : : +- CometBroadcastHashJoin (7)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.web_returns (1)
+ : : : : +- CometBroadcastExchange (6)
+ : : : : +- CometProject (5)
+ : : : : +- CometFilter (4)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : : : +- CometBroadcastExchange (11)
+ : : : +- CometFilter (10)
+ : : : +- CometScan parquet spark_catalog.default.customer_address (9)
+ : : +- BroadcastExchange (35)
+ : : +- * Filter (34)
+ : : +- * HashAggregate (33)
+ : : +- Exchange (32)
+ : : +- * HashAggregate (31)
+ : : +- * HashAggregate (30)
+ : : +- Exchange (29)
+ : : +- * ColumnarToRow (28)
+ : : +- CometHashAggregate (27)
+ : : +- CometProject (26)
+ : : +- CometBroadcastHashJoin (25)
+ : : :- CometProject (23)
+ : : : +- CometBroadcastHashJoin (22)
+ : : : :- CometFilter (20)
+ : : : : +- CometScan parquet spark_catalog.default.web_returns (19)
+ : : : +- ReusedExchange (21)
+ : : +- ReusedExchange (24)
+ : +- BroadcastExchange (41)
+ : +- * ColumnarToRow (40)
+ : +- CometFilter (39)
+ : +- CometScan parquet spark_catalog.default.customer (38)
+ +- BroadcastExchange (48)
+ +- * ColumnarToRow (47)
+ +- CometProject (46)
+ +- CometFilter (45)
+ +- CometScan parquet spark_catalog.default.customer_address (44)
+
+
+(1) Scan parquet spark_catalog.default.web_returns
+Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)]
+PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4]
+Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1))
+
+(3) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#6, d_year#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [d_date_sk#6, d_year#7]
+Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6))
+
+(5) CometProject
+Input [2]: [d_date_sk#6, d_year#7]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(6) CometBroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: [d_date_sk#6]
+
+(7) CometBroadcastHashJoin
+Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4]
+Right output [1]: [d_date_sk#6]
+Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight
+
+(8) CometProject
+Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6]
+Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3]
+
+(9) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#8, ca_state#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [2]: [ca_address_sk#8, ca_state#9]
+Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_state#9))
+
+(11) CometBroadcastExchange
+Input [2]: [ca_address_sk#8, ca_state#9]
+Arguments: [ca_address_sk#8, ca_state#9]
+
+(12) CometBroadcastHashJoin
+Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3]
+Right output [2]: [ca_address_sk#8, ca_state#9]
+Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight
+
+(13) CometProject
+Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#9]
+Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9]
+
+(14) CometHashAggregate
+Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9]
+Keys [2]: [wr_returning_customer_sk#1, ca_state#9]
+Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))]
+
+(15) ColumnarToRow [codegen id : 1]
+Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#10]
+
+(16) Exchange
+Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#10]
+Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(17) HashAggregate [codegen id : 7]
+Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#10]
+Keys [2]: [wr_returning_customer_sk#1, ca_state#9]
+Functions [1]: [sum(UnscaledValue(wr_return_amt#3))]
+Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11]
+Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14]
+
+(18) Filter [codegen id : 7]
+Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14]
+Condition : isnotnull(ctr_total_return#14)
+
+(19) Scan parquet spark_catalog.default.web_returns
+Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)]
+PushedFilters: [IsNotNull(wr_returning_addr_sk)]
+ReadSchema: struct
+
+(20) CometFilter
+Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18]
+Condition : isnotnull(wr_returning_addr_sk#16)
+
+(21) ReusedExchange [Reuses operator id: 6]
+Output [1]: [d_date_sk#20]
+
+(22) CometBroadcastHashJoin
+Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18]
+Right output [1]: [d_date_sk#20]
+Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight
+
+(23) CometProject
+Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20]
+Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17]
+
+(24) ReusedExchange [Reuses operator id: 11]
+Output [2]: [ca_address_sk#21, ca_state#22]
+
+(25) CometBroadcastHashJoin
+Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17]
+Right output [2]: [ca_address_sk#21, ca_state#22]
+Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight
+
+(26) CometProject
+Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22]
+Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22]
+
+(27) CometHashAggregate
+Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22]
+Keys [2]: [wr_returning_customer_sk#15, ca_state#22]
+Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))]
+
+(28) ColumnarToRow [codegen id : 2]
+Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23]
+
+(29) Exchange
+Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23]
+Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(30) HashAggregate [codegen id : 3]
+Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23]
+Keys [2]: [wr_returning_customer_sk#15, ca_state#22]
+Functions [1]: [sum(UnscaledValue(wr_return_amt#17))]
+Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#17))#11]
+Results [2]: [ca_state#22 AS ctr_state#24, MakeDecimal(sum(UnscaledValue(wr_return_amt#17))#11,17,2) AS ctr_total_return#25]
+
+(31) HashAggregate [codegen id : 3]
+Input [2]: [ctr_state#24, ctr_total_return#25]
+Keys [1]: [ctr_state#24]
+Functions [1]: [partial_avg(ctr_total_return#25)]
+Aggregate Attributes [2]: [sum#26, count#27]
+Results [3]: [ctr_state#24, sum#28, count#29]
+
+(32) Exchange
+Input [3]: [ctr_state#24, sum#28, count#29]
+Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(33) HashAggregate [codegen id : 4]
+Input [3]: [ctr_state#24, sum#28, count#29]
+Keys [1]: [ctr_state#24]
+Functions [1]: [avg(ctr_total_return#25)]
+Aggregate Attributes [1]: [avg(ctr_total_return#25)#30]
+Results [2]: [(avg(ctr_total_return#25)#30 * 1.2) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#24]
+
+(34) Filter [codegen id : 4]
+Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24]
+Condition : isnotnull((avg(ctr_total_return) * 1.2)#31)
+
+(35) BroadcastExchange
+Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24]
+Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4]
+
+(36) BroadcastHashJoin [codegen id : 7]
+Left keys [1]: [ctr_state#13]
+Right keys [1]: [ctr_state#24]
+Join type: Inner
+Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31)
+
+(37) Project [codegen id : 7]
+Output [2]: [ctr_customer_sk#12, ctr_total_return#14]
+Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#31, ctr_state#24]
+
+(38) Scan parquet spark_catalog.default.customer
+Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)]
+ReadSchema: struct
+
+(39) CometFilter
+Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45]
+Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34))
+
+(40) ColumnarToRow [codegen id : 5]
+Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45]
+
+(41) BroadcastExchange
+Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
+
+(42) BroadcastHashJoin [codegen id : 7]
+Left keys [1]: [ctr_customer_sk#12]
+Right keys [1]: [c_customer_sk#32]
+Join type: Inner
+Join condition: None
+
+(43) Project [codegen id : 7]
+Output [14]: [ctr_total_return#14, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45]
+Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45]
+
+(44) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#46, ca_state#47]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(45) CometFilter
+Input [2]: [ca_address_sk#46, ca_state#47]
+Condition : ((isnotnull(ca_state#47) AND (ca_state#47 = GA)) AND isnotnull(ca_address_sk#46))
+
+(46) CometProject
+Input [2]: [ca_address_sk#46, ca_state#47]
+Arguments: [ca_address_sk#46], [ca_address_sk#46]
+
+(47) ColumnarToRow [codegen id : 6]
+Input [1]: [ca_address_sk#46]
+
+(48) BroadcastExchange
+Input [1]: [ca_address_sk#46]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
+
+(49) BroadcastHashJoin [codegen id : 7]
+Left keys [1]: [c_current_addr_sk#34]
+Right keys [1]: [ca_address_sk#46]
+Join type: Inner
+Join condition: None
+
+(50) Project [codegen id : 7]
+Output [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#14]
+Input [15]: [ctr_total_return#14, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ca_address_sk#46]
+
+(51) TakeOrderedAndProject
+Input [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#14]
+Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, c_preferred_cust_flag#38 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#43 ASC NULLS FIRST, c_email_address#44 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#14]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5
+BroadcastExchange (56)
++- * ColumnarToRow (55)
+ +- CometProject (54)
+ +- CometFilter (53)
+ +- CometScan parquet spark_catalog.default.date_dim (52)
+
+
+(52) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#6, d_year#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(53) CometFilter
+Input [2]: [d_date_sk#6, d_year#7]
+Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6))
+
+(54) CometProject
+Input [2]: [d_date_sk#6, d_year#7]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(55) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#6]
+
+(56) BroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
+
+Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt
new file mode 100644
index 000000000..f24e34c36
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt
@@ -0,0 +1,77 @@
+TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return]
+ WholeStageCodegen (7)
+ Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return]
+ BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date]
+ BroadcastHashJoin [ctr_customer_sk,c_customer_sk]
+ Project [ctr_customer_sk,ctr_total_return]
+ BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)]
+ Filter [ctr_total_return]
+ HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum]
+ InputAdapter
+ Exchange [wr_returning_customer_sk,ca_state] #1
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt]
+ CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
+ CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt]
+ CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_addr_sk,wr_returning_customer_sk]
+ CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #3
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
+ CometBroadcastExchange #4
+ CometFilter [ca_address_sk,ca_state]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (4)
+ Filter [(avg(ctr_total_return) * 1.2)]
+ HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count]
+ InputAdapter
+ Exchange [ctr_state] #6
+ WholeStageCodegen (3)
+ HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count]
+ HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum]
+ InputAdapter
+ Exchange [wr_returning_customer_sk,ca_state] #7
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt]
+ CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
+ CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt]
+ CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_addr_sk]
+ CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #3
+ ReusedExchange [ca_address_sk,ca_state] #4
+ InputAdapter
+ BroadcastExchange #8
+ WholeStageCodegen (5)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [c_customer_sk,c_current_addr_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date]
+ InputAdapter
+ BroadcastExchange #9
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ca_address_sk]
+ CometFilter [ca_state,ca_address_sk]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt
new file mode 100644
index 000000000..d63ab7356
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt
@@ -0,0 +1,630 @@
+== Physical Plan ==
+* Sort (95)
++- Exchange (94)
+ +- * Project (93)
+ +- * BroadcastHashJoin Inner BuildRight (92)
+ :- * Project (78)
+ : +- * BroadcastHashJoin Inner BuildRight (77)
+ : :- * BroadcastHashJoin Inner BuildRight (63)
+ : : :- * Project (49)
+ : : : +- * BroadcastHashJoin Inner BuildRight (48)
+ : : : :- * BroadcastHashJoin Inner BuildRight (32)
+ : : : : :- * HashAggregate (16)
+ : : : : : +- Exchange (15)
+ : : : : : +- * ColumnarToRow (14)
+ : : : : : +- CometHashAggregate (13)
+ : : : : : +- CometProject (12)
+ : : : : : +- CometBroadcastHashJoin (11)
+ : : : : : :- CometProject (7)
+ : : : : : : +- CometBroadcastHashJoin (6)
+ : : : : : : :- CometFilter (2)
+ : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : : : : +- CometBroadcastExchange (5)
+ : : : : : : +- CometFilter (4)
+ : : : : : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : : : : : +- CometBroadcastExchange (10)
+ : : : : : +- CometFilter (9)
+ : : : : : +- CometScan parquet spark_catalog.default.customer_address (8)
+ : : : : +- BroadcastExchange (31)
+ : : : : +- * HashAggregate (30)
+ : : : : +- Exchange (29)
+ : : : : +- * ColumnarToRow (28)
+ : : : : +- CometHashAggregate (27)
+ : : : : +- CometProject (26)
+ : : : : +- CometBroadcastHashJoin (25)
+ : : : : :- CometProject (23)
+ : : : : : +- CometBroadcastHashJoin (22)
+ : : : : : :- CometFilter (18)
+ : : : : : : +- CometScan parquet spark_catalog.default.store_sales (17)
+ : : : : : +- CometBroadcastExchange (21)
+ : : : : : +- CometFilter (20)
+ : : : : : +- CometScan parquet spark_catalog.default.date_dim (19)
+ : : : : +- ReusedExchange (24)
+ : : : +- BroadcastExchange (47)
+ : : : +- * HashAggregate (46)
+ : : : +- Exchange (45)
+ : : : +- * ColumnarToRow (44)
+ : : : +- CometHashAggregate (43)
+ : : : +- CometProject (42)
+ : : : +- CometBroadcastHashJoin (41)
+ : : : :- CometProject (39)
+ : : : : +- CometBroadcastHashJoin (38)
+ : : : : :- CometFilter (34)
+ : : : : : +- CometScan parquet spark_catalog.default.store_sales (33)
+ : : : : +- CometBroadcastExchange (37)
+ : : : : +- CometFilter (36)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (35)
+ : : : +- ReusedExchange (40)
+ : : +- BroadcastExchange (62)
+ : : +- * HashAggregate (61)
+ : : +- Exchange (60)
+ : : +- * ColumnarToRow (59)
+ : : +- CometHashAggregate (58)
+ : : +- CometProject (57)
+ : : +- CometBroadcastHashJoin (56)
+ : : :- CometProject (54)
+ : : : +- CometBroadcastHashJoin (53)
+ : : : :- CometFilter (51)
+ : : : : +- CometScan parquet spark_catalog.default.web_sales (50)
+ : : : +- ReusedExchange (52)
+ : : +- ReusedExchange (55)
+ : +- BroadcastExchange (76)
+ : +- * HashAggregate (75)
+ : +- Exchange (74)
+ : +- * ColumnarToRow (73)
+ : +- CometHashAggregate (72)
+ : +- CometProject (71)
+ : +- CometBroadcastHashJoin (70)
+ : :- CometProject (68)
+ : : +- CometBroadcastHashJoin (67)
+ : : :- CometFilter (65)
+ : : : +- CometScan parquet spark_catalog.default.web_sales (64)
+ : : +- ReusedExchange (66)
+ : +- ReusedExchange (69)
+ +- BroadcastExchange (91)
+ +- * HashAggregate (90)
+ +- Exchange (89)
+ +- * ColumnarToRow (88)
+ +- CometHashAggregate (87)
+ +- CometProject (86)
+ +- CometBroadcastHashJoin (85)
+ :- CometProject (83)
+ : +- CometBroadcastHashJoin (82)
+ : :- CometFilter (80)
+ : : +- CometScan parquet spark_catalog.default.web_sales (79)
+ : +- ReusedExchange (81)
+ +- ReusedExchange (84)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)]
+PushedFilters: [IsNotNull(ss_addr_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3]
+Condition : isnotnull(ss_addr_sk#1)
+
+(3) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5))
+
+(5) CometBroadcastExchange
+Input [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+Arguments: [d_date_sk#5, d_year#6, d_qoy#7]
+
+(6) CometBroadcastHashJoin
+Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3]
+Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight
+
+(7) CometProject
+Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7]
+Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7]
+
+(8) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#8, ca_county#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)]
+ReadSchema: struct
+
+(9) CometFilter
+Input [2]: [ca_address_sk#8, ca_county#9]
+Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9))
+
+(10) CometBroadcastExchange
+Input [2]: [ca_address_sk#8, ca_county#9]
+Arguments: [ca_address_sk#8, ca_county#9]
+
+(11) CometBroadcastHashJoin
+Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7]
+Right output [2]: [ca_address_sk#8, ca_county#9]
+Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight
+
+(12) CometProject
+Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9]
+Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9]
+
+(13) CometHashAggregate
+Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9]
+Keys [3]: [ca_county#9, d_qoy#7, d_year#6]
+Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))]
+
+(14) ColumnarToRow [codegen id : 1]
+Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10]
+
+(15) Exchange
+Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10]
+Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(16) HashAggregate [codegen id : 12]
+Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10]
+Keys [3]: [ca_county#9, d_qoy#7, d_year#6]
+Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11]
+Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS store_sales#12]
+
+(17) Scan parquet spark_catalog.default.store_sales
+Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)]
+PushedFilters: [IsNotNull(ss_addr_sk)]
+ReadSchema: struct
+
+(18) CometFilter
+Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15]
+Condition : isnotnull(ss_addr_sk#13)
+
+(19) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(20) CometFilter
+Input [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17))
+
+(21) CometBroadcastExchange
+Input [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+Arguments: [d_date_sk#17, d_year#18, d_qoy#19]
+
+(22) CometBroadcastHashJoin
+Left output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15]
+Right output [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight
+
+(23) CometProject
+Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_year#18, d_qoy#19]
+Arguments: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19], [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19]
+
+(24) ReusedExchange [Reuses operator id: 10]
+Output [2]: [ca_address_sk#20, ca_county#21]
+
+(25) CometBroadcastHashJoin
+Left output [4]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19]
+Right output [2]: [ca_address_sk#20, ca_county#21]
+Arguments: [ss_addr_sk#13], [ca_address_sk#20], Inner, BuildRight
+
+(26) CometProject
+Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_address_sk#20, ca_county#21]
+Arguments: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21], [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21]
+
+(27) CometHashAggregate
+Input [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21]
+Keys [3]: [ca_county#21, d_qoy#19, d_year#18]
+Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#14))]
+
+(28) ColumnarToRow [codegen id : 2]
+Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22]
+
+(29) Exchange
+Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22]
+Arguments: hashpartitioning(ca_county#21, d_qoy#19, d_year#18, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(30) HashAggregate [codegen id : 3]
+Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22]
+Keys [3]: [ca_county#21, d_qoy#19, d_year#18]
+Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#14))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#14))#11]
+Results [2]: [ca_county#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#14))#11,17,2) AS store_sales#23]
+
+(31) BroadcastExchange
+Input [2]: [ca_county#21, store_sales#23]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3]
+
+(32) BroadcastHashJoin [codegen id : 12]
+Left keys [1]: [ca_county#9]
+Right keys [1]: [ca_county#21]
+Join type: Inner
+Join condition: None
+
+(33) Scan parquet spark_catalog.default.store_sales
+Output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)]
+PushedFilters: [IsNotNull(ss_addr_sk)]
+ReadSchema: struct
+
+(34) CometFilter
+Input [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26]
+Condition : isnotnull(ss_addr_sk#24)
+
+(35) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(36) CometFilter
+Input [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28))
+
+(37) CometBroadcastExchange
+Input [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+Arguments: [d_date_sk#28, d_year#29, d_qoy#30]
+
+(38) CometBroadcastHashJoin
+Left output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26]
+Right output [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight
+
+(39) CometProject
+Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29, d_qoy#30]
+Arguments: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30], [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30]
+
+(40) ReusedExchange [Reuses operator id: 10]
+Output [2]: [ca_address_sk#31, ca_county#32]
+
+(41) CometBroadcastHashJoin
+Left output [4]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30]
+Right output [2]: [ca_address_sk#31, ca_county#32]
+Arguments: [ss_addr_sk#24], [ca_address_sk#31], Inner, BuildRight
+
+(42) CometProject
+Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_address_sk#31, ca_county#32]
+Arguments: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32], [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32]
+
+(43) CometHashAggregate
+Input [4]: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32]
+Keys [3]: [ca_county#32, d_qoy#30, d_year#29]
+Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#25))]
+
+(44) ColumnarToRow [codegen id : 4]
+Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33]
+
+(45) Exchange
+Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33]
+Arguments: hashpartitioning(ca_county#32, d_qoy#30, d_year#29, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+
+(46) HashAggregate [codegen id : 5]
+Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33]
+Keys [3]: [ca_county#32, d_qoy#30, d_year#29]
+Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#25))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#25))#11]
+Results [2]: [ca_county#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#25))#11,17,2) AS store_sales#34]
+
+(47) BroadcastExchange
+Input [2]: [ca_county#32, store_sales#34]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5]
+
+(48) BroadcastHashJoin [codegen id : 12]
+Left keys [1]: [ca_county#21]
+Right keys [1]: [ca_county#32]
+Join type: Inner
+Join condition: None
+
+(49) Project [codegen id : 12]
+Output [5]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34]
+Input [7]: [ca_county#9, d_year#6, store_sales#12, ca_county#21, store_sales#23, ca_county#32, store_sales#34]
+
+(50) Scan parquet spark_catalog.default.web_sales
+Output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)]
+PushedFilters: [IsNotNull(ws_bill_addr_sk)]
+ReadSchema: struct
+
+(51) CometFilter
+Input [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37]
+Condition : isnotnull(ws_bill_addr_sk#35)
+
+(52) ReusedExchange [Reuses operator id: 5]
+Output [3]: [d_date_sk#39, d_year#40, d_qoy#41]
+
+(53) CometBroadcastHashJoin
+Left output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37]
+Right output [3]: [d_date_sk#39, d_year#40, d_qoy#41]
+Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner, BuildRight
+
+(54) CometProject
+Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41]
+Arguments: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41], [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41]
+
+(55) ReusedExchange [Reuses operator id: 10]
+Output [2]: [ca_address_sk#42, ca_county#43]
+
+(56) CometBroadcastHashJoin
+Left output [4]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41]
+Right output [2]: [ca_address_sk#42, ca_county#43]
+Arguments: [ws_bill_addr_sk#35], [ca_address_sk#42], Inner, BuildRight
+
+(57) CometProject
+Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#42, ca_county#43]
+Arguments: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43], [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43]
+
+(58) CometHashAggregate
+Input [4]: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43]
+Keys [3]: [ca_county#43, d_qoy#41, d_year#40]
+Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#36))]
+
+(59) ColumnarToRow [codegen id : 6]
+Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44]
+
+(60) Exchange
+Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44]
+Arguments: hashpartitioning(ca_county#43, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+
+(61) HashAggregate [codegen id : 7]
+Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44]
+Keys [3]: [ca_county#43, d_qoy#41, d_year#40]
+Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#36))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#36))#45]
+Results [2]: [ca_county#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#36))#45,17,2) AS web_sales#46]
+
+(62) BroadcastExchange
+Input [2]: [ca_county#43, web_sales#46]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7]
+
+(63) BroadcastHashJoin [codegen id : 12]
+Left keys [1]: [ca_county#9]
+Right keys [1]: [ca_county#43]
+Join type: Inner
+Join condition: None
+
+(64) Scan parquet spark_catalog.default.web_sales
+Output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)]
+PushedFilters: [IsNotNull(ws_bill_addr_sk)]
+ReadSchema: struct
+
+(65) CometFilter
+Input [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49]
+Condition : isnotnull(ws_bill_addr_sk#47)
+
+(66) ReusedExchange [Reuses operator id: 21]
+Output [3]: [d_date_sk#51, d_year#52, d_qoy#53]
+
+(67) CometBroadcastHashJoin
+Left output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49]
+Right output [3]: [d_date_sk#51, d_year#52, d_qoy#53]
+Arguments: [ws_sold_date_sk#49], [d_date_sk#51], Inner, BuildRight
+
+(68) CometProject
+Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49, d_date_sk#51, d_year#52, d_qoy#53]
+Arguments: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53], [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53]
+
+(69) ReusedExchange [Reuses operator id: 10]
+Output [2]: [ca_address_sk#54, ca_county#55]
+
+(70) CometBroadcastHashJoin
+Left output [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53]
+Right output [2]: [ca_address_sk#54, ca_county#55]
+Arguments: [ws_bill_addr_sk#47], [ca_address_sk#54], Inner, BuildRight
+
+(71) CometProject
+Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_address_sk#54, ca_county#55]
+Arguments: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55], [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55]
+
+(72) CometHashAggregate
+Input [4]: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55]
+Keys [3]: [ca_county#55, d_qoy#53, d_year#52]
+Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#48))]
+
+(73) ColumnarToRow [codegen id : 8]
+Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56]
+
+(74) Exchange
+Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56]
+Arguments: hashpartitioning(ca_county#55, d_qoy#53, d_year#52, 5), ENSURE_REQUIREMENTS, [plan_id=8]
+
+(75) HashAggregate [codegen id : 9]
+Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56]
+Keys [3]: [ca_county#55, d_qoy#53, d_year#52]
+Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#48))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#48))#45]
+Results [2]: [ca_county#55, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#48))#45,17,2) AS web_sales#57]
+
+(76) BroadcastExchange
+Input [2]: [ca_county#55, web_sales#57]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9]
+
+(77) BroadcastHashJoin [codegen id : 12]
+Left keys [1]: [ca_county#43]
+Right keys [1]: [ca_county#55]
+Join type: Inner
+Join condition: (CASE WHEN (web_sales#46 > 0.00) THEN (web_sales#57 / web_sales#46) END > CASE WHEN (store_sales#12 > 0.00) THEN (store_sales#23 / store_sales#12) END)
+
+(78) Project [codegen id : 12]
+Output [8]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57]
+Input [9]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, ca_county#55, web_sales#57]
+
+(79) Scan parquet spark_catalog.default.web_sales
+Output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)]
+PushedFilters: [IsNotNull(ws_bill_addr_sk)]
+ReadSchema: struct
+
+(80) CometFilter
+Input [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60]
+Condition : isnotnull(ws_bill_addr_sk#58)
+
+(81) ReusedExchange [Reuses operator id: 37]
+Output [3]: [d_date_sk#62, d_year#63, d_qoy#64]
+
+(82) CometBroadcastHashJoin
+Left output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60]
+Right output [3]: [d_date_sk#62, d_year#63, d_qoy#64]
+Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight
+
+(83) CometProject
+Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63, d_qoy#64]
+Arguments: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64], [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64]
+
+(84) ReusedExchange [Reuses operator id: 10]
+Output [2]: [ca_address_sk#65, ca_county#66]
+
+(85) CometBroadcastHashJoin
+Left output [4]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64]
+Right output [2]: [ca_address_sk#65, ca_county#66]
+Arguments: [ws_bill_addr_sk#58], [ca_address_sk#65], Inner, BuildRight
+
+(86) CometProject
+Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_address_sk#65, ca_county#66]
+Arguments: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66], [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66]
+
+(87) CometHashAggregate
+Input [4]: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66]
+Keys [3]: [ca_county#66, d_qoy#64, d_year#63]
+Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#59))]
+
+(88) ColumnarToRow [codegen id : 10]
+Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67]
+
+(89) Exchange
+Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67]
+Arguments: hashpartitioning(ca_county#66, d_qoy#64, d_year#63, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+
+(90) HashAggregate [codegen id : 11]
+Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67]
+Keys [3]: [ca_county#66, d_qoy#64, d_year#63]
+Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#59))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#59))#45]
+Results [2]: [ca_county#66, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#45,17,2) AS web_sales#68]
+
+(91) BroadcastExchange
+Input [2]: [ca_county#66, web_sales#68]
+Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11]
+
+(92) BroadcastHashJoin [codegen id : 12]
+Left keys [1]: [ca_county#43]
+Right keys [1]: [ca_county#66]
+Join type: Inner
+Join condition: (CASE WHEN (web_sales#57 > 0.00) THEN (web_sales#68 / web_sales#57) END > CASE WHEN (store_sales#23 > 0.00) THEN (store_sales#34 / store_sales#23) END)
+
+(93) Project [codegen id : 12]
+Output [6]: [ca_county#9, d_year#6, (web_sales#57 / web_sales#46) AS web_q1_q2_increase#69, (store_sales#23 / store_sales#12) AS store_q1_q2_increase#70, (web_sales#68 / web_sales#57) AS web_q2_q3_increase#71, (store_sales#34 / store_sales#23) AS store_q2_q3_increase#72]
+Input [10]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57, ca_county#66, web_sales#68]
+
+(94) Exchange
+Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72]
+Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=12]
+
+(95) Sort [codegen id : 13]
+Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72]
+Arguments: [ca_county#9 ASC NULLS FIRST], true, 0
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4
+BroadcastExchange (99)
++- * ColumnarToRow (98)
+ +- CometFilter (97)
+ +- CometScan parquet spark_catalog.default.date_dim (96)
+
+
+(96) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(97) CometFilter
+Input [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5))
+
+(98) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+
+(99) BroadcastExchange
+Input [3]: [d_date_sk#5, d_year#6, d_qoy#7]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13]
+
+Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#16
+BroadcastExchange (103)
++- * ColumnarToRow (102)
+ +- CometFilter (101)
+ +- CometScan parquet spark_catalog.default.date_dim (100)
+
+
+(100) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(101) CometFilter
+Input [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17))
+
+(102) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+
+(103) BroadcastExchange
+Input [3]: [d_date_sk#17, d_year#18, d_qoy#19]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14]
+
+Subquery:3 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27
+BroadcastExchange (107)
++- * ColumnarToRow (106)
+ +- CometFilter (105)
+ +- CometScan parquet spark_catalog.default.date_dim (104)
+
+
+(104) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(105) CometFilter
+Input [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28))
+
+(106) ColumnarToRow [codegen id : 1]
+Input [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+
+(107) BroadcastExchange
+Input [3]: [d_date_sk#28, d_year#29, d_qoy#30]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15]
+
+Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4
+
+Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16
+
+Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt
new file mode 100644
index 000000000..a94a8a94d
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt
@@ -0,0 +1,150 @@
+WholeStageCodegen (13)
+ Sort [ca_county]
+ InputAdapter
+ Exchange [ca_county] #1
+ WholeStageCodegen (12)
+ Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales]
+ BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales]
+ Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales]
+ BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales]
+ BroadcastHashJoin [ca_county,ca_county]
+ Project [ca_county,d_year,store_sales,store_sales,store_sales]
+ BroadcastHashJoin [ca_county,ca_county]
+ BroadcastHashJoin [ca_county,ca_county]
+ HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum]
+ InputAdapter
+ Exchange [ca_county,d_qoy,d_year] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
+ CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_addr_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ CometBroadcastExchange #4
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ CometBroadcastExchange #5
+ CometFilter [ca_address_sk,ca_county]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (3)
+ HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum]
+ InputAdapter
+ Exchange [ca_county,d_qoy,d_year] #7
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
+ CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_addr_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #2
+ BroadcastExchange #8
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ CometBroadcastExchange #9
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ ReusedExchange [ca_address_sk,ca_county] #5
+ InputAdapter
+ BroadcastExchange #10
+ WholeStageCodegen (5)
+ HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum]
+ InputAdapter
+ Exchange [ca_county,d_qoy,d_year] #11
+ WholeStageCodegen (4)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
+ CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_addr_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #3
+ BroadcastExchange #12
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ CometBroadcastExchange #13
+ CometFilter [d_qoy,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ ReusedExchange [ca_address_sk,ca_county] #5
+ InputAdapter
+ BroadcastExchange #14
+ WholeStageCodegen (7)
+ HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum]
+ InputAdapter
+ Exchange [ca_county,d_qoy,d_year] #15
+ WholeStageCodegen (6)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_bill_addr_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk,d_year,d_qoy] #4
+ ReusedExchange [ca_address_sk,ca_county] #5
+ InputAdapter
+ BroadcastExchange #16
+ WholeStageCodegen (9)
+ HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum]
+ InputAdapter
+ Exchange [ca_county,d_qoy,d_year] #17
+ WholeStageCodegen (8)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_bill_addr_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #2
+ ReusedExchange [d_date_sk,d_year,d_qoy] #9
+ ReusedExchange [ca_address_sk,ca_county] #5
+ InputAdapter
+ BroadcastExchange #18
+ WholeStageCodegen (11)
+ HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum]
+ InputAdapter
+ Exchange [ca_county,d_qoy,d_year] #19
+ WholeStageCodegen (10)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_bill_addr_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #3
+ ReusedExchange [d_date_sk,d_year,d_qoy] #13
+ ReusedExchange [ca_address_sk,ca_county] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt
new file mode 100644
index 000000000..b7eee60b1
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt
@@ -0,0 +1,220 @@
+== Physical Plan ==
+* HashAggregate (31)
++- Exchange (30)
+ +- * HashAggregate (29)
+ +- * Project (28)
+ +- * BroadcastHashJoin Inner BuildRight (27)
+ :- * Project (25)
+ : +- * BroadcastHashJoin Inner BuildRight (24)
+ : :- * ColumnarToRow (9)
+ : : +- CometProject (8)
+ : : +- CometBroadcastHashJoin (7)
+ : : :- CometFilter (2)
+ : : : +- CometScan parquet spark_catalog.default.catalog_sales (1)
+ : : +- CometBroadcastExchange (6)
+ : : +- CometProject (5)
+ : : +- CometFilter (4)
+ : : +- CometScan parquet spark_catalog.default.item (3)
+ : +- BroadcastExchange (23)
+ : +- * Filter (22)
+ : +- * HashAggregate (21)
+ : +- Exchange (20)
+ : +- * ColumnarToRow (19)
+ : +- CometHashAggregate (18)
+ : +- CometProject (17)
+ : +- CometBroadcastHashJoin (16)
+ : :- CometFilter (11)
+ : : +- CometScan parquet spark_catalog.default.catalog_sales (10)
+ : +- CometBroadcastExchange (15)
+ : +- CometProject (14)
+ : +- CometFilter (13)
+ : +- CometScan parquet spark_catalog.default.date_dim (12)
+ +- ReusedExchange (26)
+
+
+(1) Scan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)]
+PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3]
+Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2))
+
+(3) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#5, i_manufact_id#6]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [2]: [i_item_sk#5, i_manufact_id#6]
+Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5))
+
+(5) CometProject
+Input [2]: [i_item_sk#5, i_manufact_id#6]
+Arguments: [i_item_sk#5], [i_item_sk#5]
+
+(6) CometBroadcastExchange
+Input [1]: [i_item_sk#5]
+Arguments: [i_item_sk#5]
+
+(7) CometBroadcastHashJoin
+Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3]
+Right output [1]: [i_item_sk#5]
+Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight
+
+(8) CometProject
+Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5]
+Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5]
+
+(9) ColumnarToRow [codegen id : 4]
+Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5]
+
+(10) Scan parquet spark_catalog.default.catalog_sales
+Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)]
+PushedFilters: [IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(11) CometFilter
+Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9]
+Condition : isnotnull(cs_item_sk#7)
+
+(12) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#11, d_date#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(13) CometFilter
+Input [2]: [d_date_sk#11, d_date#12]
+Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11))
+
+(14) CometProject
+Input [2]: [d_date_sk#11, d_date#12]
+Arguments: [d_date_sk#11], [d_date_sk#11]
+
+(15) CometBroadcastExchange
+Input [1]: [d_date_sk#11]
+Arguments: [d_date_sk#11]
+
+(16) CometBroadcastHashJoin
+Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9]
+Right output [1]: [d_date_sk#11]
+Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight
+
+(17) CometProject
+Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11]
+Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8]
+
+(18) CometHashAggregate
+Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8]
+Keys [1]: [cs_item_sk#7]
+Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))]
+
+(19) ColumnarToRow [codegen id : 1]
+Input [3]: [cs_item_sk#7, sum#13, count#14]
+
+(20) Exchange
+Input [3]: [cs_item_sk#7, sum#13, count#14]
+Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(21) HashAggregate [codegen id : 2]
+Input [3]: [cs_item_sk#7, sum#13, count#14]
+Keys [1]: [cs_item_sk#7]
+Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))]
+Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15]
+Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7]
+
+(22) Filter [codegen id : 2]
+Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7]
+Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16)
+
+(23) BroadcastExchange
+Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7]
+Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=2]
+
+(24) BroadcastHashJoin [codegen id : 4]
+Left keys [1]: [i_item_sk#5]
+Right keys [1]: [cs_item_sk#7]
+Join type: Inner
+Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16)
+
+(25) Project [codegen id : 4]
+Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3]
+Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7]
+
+(26) ReusedExchange [Reuses operator id: 36]
+Output [1]: [d_date_sk#17]
+
+(27) BroadcastHashJoin [codegen id : 4]
+Left keys [1]: [cs_sold_date_sk#3]
+Right keys [1]: [d_date_sk#17]
+Join type: Inner
+Join condition: None
+
+(28) Project [codegen id : 4]
+Output [1]: [cs_ext_discount_amt#2]
+Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17]
+
+(29) HashAggregate [codegen id : 4]
+Input [1]: [cs_ext_discount_amt#2]
+Keys: []
+Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))]
+Aggregate Attributes [1]: [sum#18]
+Results [1]: [sum#19]
+
+(30) Exchange
+Input [1]: [sum#19]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3]
+
+(31) HashAggregate [codegen id : 5]
+Input [1]: [sum#19]
+Keys: []
+Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))]
+Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#20]
+Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#20,17,2) AS excess discount amount#21]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4
+BroadcastExchange (36)
++- * ColumnarToRow (35)
+ +- CometProject (34)
+ +- CometFilter (33)
+ +- CometScan parquet spark_catalog.default.date_dim (32)
+
+
+(32) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#17, d_date#22]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(33) CometFilter
+Input [2]: [d_date_sk#17, d_date#22]
+Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#17))
+
+(34) CometProject
+Input [2]: [d_date_sk#17, d_date#22]
+Arguments: [d_date_sk#17], [d_date_sk#17]
+
+(35) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#17]
+
+(36) BroadcastExchange
+Input [1]: [d_date_sk#17]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
+
+Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt
new file mode 100644
index 000000000..b8df1e929
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt
@@ -0,0 +1,50 @@
+WholeStageCodegen (5)
+ HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum]
+ InputAdapter
+ Exchange #1
+ WholeStageCodegen (4)
+ HashAggregate [cs_ext_discount_amt] [sum,sum]
+ Project [cs_ext_discount_amt]
+ BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ Project [cs_ext_discount_amt,cs_sold_date_sk]
+ BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))]
+ ColumnarToRow
+ InputAdapter
+ CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometFilter [cs_item_sk,cs_ext_discount_amt]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
+ CometBroadcastExchange #3
+ CometProject [i_item_sk]
+ CometFilter [i_manufact_id,i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
+ InputAdapter
+ BroadcastExchange #4
+ WholeStageCodegen (2)
+ Filter [(1.3 * avg(cs_ext_discount_amt))]
+ HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count]
+ InputAdapter
+ Exchange [cs_item_sk] #5
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [cs_item_sk,cs_ext_discount_amt]
+ CometProject [cs_item_sk,cs_ext_discount_amt]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ CometBroadcastExchange #6
+ CometProject [d_date_sk]
+ CometFilter [d_date,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
+ InputAdapter
+ ReusedExchange [d_date_sk] #2
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt
new file mode 100644
index 000000000..8e385ed4e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt
@@ -0,0 +1,396 @@
+== Physical Plan ==
+TakeOrderedAndProject (63)
++- * HashAggregate (62)
+ +- Exchange (61)
+ +- * HashAggregate (60)
+ +- Union (59)
+ :- * HashAggregate (28)
+ : +- Exchange (27)
+ : +- * ColumnarToRow (26)
+ : +- CometHashAggregate (25)
+ : +- CometProject (24)
+ : +- CometBroadcastHashJoin (23)
+ : :- CometProject (14)
+ : : +- CometBroadcastHashJoin (13)
+ : : :- CometProject (8)
+ : : : +- CometBroadcastHashJoin (7)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : +- CometBroadcastExchange (6)
+ : : : +- CometProject (5)
+ : : : +- CometFilter (4)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : : +- CometBroadcastExchange (12)
+ : : +- CometProject (11)
+ : : +- CometFilter (10)
+ : : +- CometScan parquet spark_catalog.default.customer_address (9)
+ : +- CometBroadcastExchange (22)
+ : +- CometBroadcastHashJoin (21)
+ : :- CometFilter (16)
+ : : +- CometScan parquet spark_catalog.default.item (15)
+ : +- CometBroadcastExchange (20)
+ : +- CometProject (19)
+ : +- CometFilter (18)
+ : +- CometScan parquet spark_catalog.default.item (17)
+ :- * HashAggregate (43)
+ : +- Exchange (42)
+ : +- * ColumnarToRow (41)
+ : +- CometHashAggregate (40)
+ : +- CometProject (39)
+ : +- CometBroadcastHashJoin (38)
+ : :- CometProject (36)
+ : : +- CometBroadcastHashJoin (35)
+ : : :- CometProject (33)
+ : : : +- CometBroadcastHashJoin (32)
+ : : : :- CometFilter (30)
+ : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29)
+ : : : +- ReusedExchange (31)
+ : : +- ReusedExchange (34)
+ : +- ReusedExchange (37)
+ +- * HashAggregate (58)
+ +- Exchange (57)
+ +- * ColumnarToRow (56)
+ +- CometHashAggregate (55)
+ +- CometProject (54)
+ +- CometBroadcastHashJoin (53)
+ :- CometProject (51)
+ : +- CometBroadcastHashJoin (50)
+ : :- CometProject (48)
+ : : +- CometBroadcastHashJoin (47)
+ : : :- CometFilter (45)
+ : : : +- CometScan parquet spark_catalog.default.web_sales (44)
+ : : +- ReusedExchange (46)
+ : +- ReusedExchange (49)
+ +- ReusedExchange (52)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)]
+PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4]
+Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1))
+
+(3) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
+Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6))
+
+(5) CometProject
+Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(6) CometBroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: [d_date_sk#6]
+
+(7) CometBroadcastHashJoin
+Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4]
+Right output [1]: [d_date_sk#6]
+Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight
+
+(8) CometProject
+Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6]
+Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3]
+
+(9) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#9, ca_gmt_offset#10]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [2]: [ca_address_sk#9, ca_gmt_offset#10]
+Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9))
+
+(11) CometProject
+Input [2]: [ca_address_sk#9, ca_gmt_offset#10]
+Arguments: [ca_address_sk#9], [ca_address_sk#9]
+
+(12) CometBroadcastExchange
+Input [1]: [ca_address_sk#9]
+Arguments: [ca_address_sk#9]
+
+(13) CometBroadcastHashJoin
+Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3]
+Right output [1]: [ca_address_sk#9]
+Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight
+
+(14) CometProject
+Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9]
+Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3]
+
+(15) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#11, i_manufact_id#12]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_item_sk)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [2]: [i_item_sk#11, i_manufact_id#12]
+Condition : isnotnull(i_item_sk#11)
+
+(17) Scan parquet spark_catalog.default.item
+Output [2]: [i_category#13, i_manufact_id#14]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/item]
+PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )]
+ReadSchema: struct
+
+(18) CometFilter
+Input [2]: [i_category#13, i_manufact_id#14]
+Condition : (isnotnull(i_category#13) AND (i_category#13 = Electronics ))
+
+(19) CometProject
+Input [2]: [i_category#13, i_manufact_id#14]
+Arguments: [i_manufact_id#14], [i_manufact_id#14]
+
+(20) CometBroadcastExchange
+Input [1]: [i_manufact_id#14]
+Arguments: [i_manufact_id#14]
+
+(21) CometBroadcastHashJoin
+Left output [2]: [i_item_sk#11, i_manufact_id#12]
+Right output [1]: [i_manufact_id#14]
+Arguments: [i_manufact_id#12], [i_manufact_id#14], LeftSemi, BuildRight
+
+(22) CometBroadcastExchange
+Input [2]: [i_item_sk#11, i_manufact_id#12]
+Arguments: [i_item_sk#11, i_manufact_id#12]
+
+(23) CometBroadcastHashJoin
+Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3]
+Right output [2]: [i_item_sk#11, i_manufact_id#12]
+Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight
+
+(24) CometProject
+Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12]
+Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12]
+
+(25) CometHashAggregate
+Input [2]: [ss_ext_sales_price#3, i_manufact_id#12]
+Keys [1]: [i_manufact_id#12]
+Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))]
+
+(26) ColumnarToRow [codegen id : 1]
+Input [2]: [i_manufact_id#12, sum#15]
+
+(27) Exchange
+Input [2]: [i_manufact_id#12, sum#15]
+Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(28) HashAggregate [codegen id : 2]
+Input [2]: [i_manufact_id#12, sum#15]
+Keys [1]: [i_manufact_id#12]
+Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16]
+Results [2]: [i_manufact_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS total_sales#17]
+
+(29) Scan parquet spark_catalog.default.catalog_sales
+Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)]
+PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)]
+ReadSchema: struct
+
+(30) CometFilter
+Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21]
+Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19))
+
+(31) ReusedExchange [Reuses operator id: 6]
+Output [1]: [d_date_sk#23]
+
+(32) CometBroadcastHashJoin
+Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21]
+Right output [1]: [d_date_sk#23]
+Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight
+
+(33) CometProject
+Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23]
+Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20]
+
+(34) ReusedExchange [Reuses operator id: 12]
+Output [1]: [ca_address_sk#24]
+
+(35) CometBroadcastHashJoin
+Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20]
+Right output [1]: [ca_address_sk#24]
+Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight
+
+(36) CometProject
+Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24]
+Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20]
+
+(37) ReusedExchange [Reuses operator id: 22]
+Output [2]: [i_item_sk#25, i_manufact_id#26]
+
+(38) CometBroadcastHashJoin
+Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20]
+Right output [2]: [i_item_sk#25, i_manufact_id#26]
+Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight
+
+(39) CometProject
+Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_manufact_id#26]
+Arguments: [cs_ext_sales_price#20, i_manufact_id#26], [cs_ext_sales_price#20, i_manufact_id#26]
+
+(40) CometHashAggregate
+Input [2]: [cs_ext_sales_price#20, i_manufact_id#26]
+Keys [1]: [i_manufact_id#26]
+Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))]
+
+(41) ColumnarToRow [codegen id : 3]
+Input [2]: [i_manufact_id#26, sum#27]
+
+(42) Exchange
+Input [2]: [i_manufact_id#26, sum#27]
+Arguments: hashpartitioning(i_manufact_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2]
+
+(43) HashAggregate [codegen id : 4]
+Input [2]: [i_manufact_id#26, sum#27]
+Keys [1]: [i_manufact_id#26]
+Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))]
+Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28]
+Results [2]: [i_manufact_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29]
+
+(44) Scan parquet spark_catalog.default.web_sales
+Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)]
+PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)]
+ReadSchema: struct
+
+(45) CometFilter
+Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33]
+Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30))
+
+(46) ReusedExchange [Reuses operator id: 6]
+Output [1]: [d_date_sk#35]
+
+(47) CometBroadcastHashJoin
+Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33]
+Right output [1]: [d_date_sk#35]
+Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight
+
+(48) CometProject
+Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35]
+Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32]
+
+(49) ReusedExchange [Reuses operator id: 12]
+Output [1]: [ca_address_sk#36]
+
+(50) CometBroadcastHashJoin
+Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32]
+Right output [1]: [ca_address_sk#36]
+Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight
+
+(51) CometProject
+Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36]
+Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32]
+
+(52) ReusedExchange [Reuses operator id: 22]
+Output [2]: [i_item_sk#37, i_manufact_id#38]
+
+(53) CometBroadcastHashJoin
+Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32]
+Right output [2]: [i_item_sk#37, i_manufact_id#38]
+Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight
+
+(54) CometProject
+Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38]
+Arguments: [ws_ext_sales_price#32, i_manufact_id#38], [ws_ext_sales_price#32, i_manufact_id#38]
+
+(55) CometHashAggregate
+Input [2]: [ws_ext_sales_price#32, i_manufact_id#38]
+Keys [1]: [i_manufact_id#38]
+Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))]
+
+(56) ColumnarToRow [codegen id : 5]
+Input [2]: [i_manufact_id#38, sum#39]
+
+(57) Exchange
+Input [2]: [i_manufact_id#38, sum#39]
+Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(58) HashAggregate [codegen id : 6]
+Input [2]: [i_manufact_id#38, sum#39]
+Keys [1]: [i_manufact_id#38]
+Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))]
+Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#40]
+Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41]
+
+(59) Union
+
+(60) HashAggregate [codegen id : 7]
+Input [2]: [i_manufact_id#12, total_sales#17]
+Keys [1]: [i_manufact_id#12]
+Functions [1]: [partial_sum(total_sales#17)]
+Aggregate Attributes [2]: [sum#42, isEmpty#43]
+Results [3]: [i_manufact_id#12, sum#44, isEmpty#45]
+
+(61) Exchange
+Input [3]: [i_manufact_id#12, sum#44, isEmpty#45]
+Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+
+(62) HashAggregate [codegen id : 8]
+Input [3]: [i_manufact_id#12, sum#44, isEmpty#45]
+Keys [1]: [i_manufact_id#12]
+Functions [1]: [sum(total_sales#17)]
+Aggregate Attributes [1]: [sum(total_sales#17)#46]
+Results [2]: [i_manufact_id#12, sum(total_sales#17)#46 AS total_sales#47]
+
+(63) TakeOrderedAndProject
+Input [2]: [i_manufact_id#12, total_sales#47]
+Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#12, total_sales#47]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
+BroadcastExchange (68)
++- * ColumnarToRow (67)
+ +- CometProject (66)
+ +- CometFilter (65)
+ +- CometScan parquet spark_catalog.default.date_dim (64)
+
+
+(64) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#6, d_year#7, d_moy#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(65) CometFilter
+Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
+Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6))
+
+(66) CometProject
+Input [3]: [d_date_sk#6, d_year#7, d_moy#8]
+Arguments: [d_date_sk#6], [d_date_sk#6]
+
+(67) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#6]
+
+(68) BroadcastExchange
+Input [1]: [d_date_sk#6]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5]
+
+Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5
+
+Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt
new file mode 100644
index 000000000..af2b7cb5d
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt
@@ -0,0 +1,89 @@
+TakeOrderedAndProject [total_sales,i_manufact_id]
+ WholeStageCodegen (8)
+ HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty]
+ InputAdapter
+ Exchange [i_manufact_id] #1
+ WholeStageCodegen (7)
+ HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty]
+ InputAdapter
+ Union
+ WholeStageCodegen (2)
+ HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum]
+ InputAdapter
+ Exchange [i_manufact_id] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_manufact_id,ss_ext_sales_price]
+ CometProject [ss_ext_sales_price,i_manufact_id]
+ CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometProject [ss_item_sk,ss_ext_sales_price]
+ CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_addr_sk,ss_item_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #4
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_moy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
+ CometBroadcastExchange #5
+ CometProject [ca_address_sk]
+ CometFilter [ca_gmt_offset,ca_address_sk]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
+ CometBroadcastExchange #6
+ CometBroadcastHashJoin [i_manufact_id,i_manufact_id]
+ CometFilter [i_item_sk]
+ CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
+ CometBroadcastExchange #7
+ CometProject [i_manufact_id]
+ CometFilter [i_category]
+ CometScan parquet spark_catalog.default.item [i_category,i_manufact_id]
+ WholeStageCodegen (4)
+ HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum]
+ InputAdapter
+ Exchange [i_manufact_id] #8
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_manufact_id,cs_ext_sales_price]
+ CometProject [cs_ext_sales_price,i_manufact_id]
+ CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometProject [cs_item_sk,cs_ext_sales_price]
+ CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #4
+ ReusedExchange [ca_address_sk] #5
+ ReusedExchange [i_item_sk,i_manufact_id] #6
+ WholeStageCodegen (6)
+ HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum]
+ InputAdapter
+ Exchange [i_manufact_id] #9
+ WholeStageCodegen (5)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [i_manufact_id,ws_ext_sales_price]
+ CometProject [ws_ext_sales_price,i_manufact_id]
+ CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometProject [ws_item_sk,ws_ext_sales_price]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_bill_addr_sk,ws_item_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #4
+ ReusedExchange [ca_address_sk] #5
+ ReusedExchange [i_item_sk,i_manufact_id] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt
new file mode 100644
index 000000000..5609dc11c
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt
@@ -0,0 +1,224 @@
+== Physical Plan ==
+* Sort (33)
++- Exchange (32)
+ +- * Project (31)
+ +- * BroadcastHashJoin Inner BuildRight (30)
+ :- * Filter (25)
+ : +- * HashAggregate (24)
+ : +- Exchange (23)
+ : +- * ColumnarToRow (22)
+ : +- CometHashAggregate (21)
+ : +- CometProject (20)
+ : +- CometBroadcastHashJoin (19)
+ : :- CometProject (14)
+ : : +- CometBroadcastHashJoin (13)
+ : : :- CometProject (8)
+ : : : +- CometBroadcastHashJoin (7)
+ : : : :- CometFilter (2)
+ : : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : : +- CometBroadcastExchange (6)
+ : : : +- CometProject (5)
+ : : : +- CometFilter (4)
+ : : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : : +- CometBroadcastExchange (12)
+ : : +- CometProject (11)
+ : : +- CometFilter (10)
+ : : +- CometScan parquet spark_catalog.default.store (9)
+ : +- CometBroadcastExchange (18)
+ : +- CometProject (17)
+ : +- CometFilter (16)
+ : +- CometScan parquet spark_catalog.default.household_demographics (15)
+ +- BroadcastExchange (29)
+ +- * ColumnarToRow (28)
+ +- CometFilter (27)
+ +- CometScan parquet spark_catalog.default.customer (26)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)]
+PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5]
+Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1))
+
+(3) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#7, d_year#8, d_dom#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(4) CometFilter
+Input [3]: [d_date_sk#7, d_year#8, d_dom#9]
+Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7))
+
+(5) CometProject
+Input [3]: [d_date_sk#7, d_year#8, d_dom#9]
+Arguments: [d_date_sk#7], [d_date_sk#7]
+
+(6) CometBroadcastExchange
+Input [1]: [d_date_sk#7]
+Arguments: [d_date_sk#7]
+
+(7) CometBroadcastHashJoin
+Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5]
+Right output [1]: [d_date_sk#7]
+Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight
+
+(8) CometProject
+Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7]
+Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4]
+
+(9) Scan parquet spark_catalog.default.store
+Output [2]: [s_store_sk#10, s_county#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/store]
+PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)]
+ReadSchema: struct
+
+(10) CometFilter
+Input [2]: [s_store_sk#10, s_county#11]
+Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10))
+
+(11) CometProject
+Input [2]: [s_store_sk#10, s_county#11]
+Arguments: [s_store_sk#10], [s_store_sk#10]
+
+(12) CometBroadcastExchange
+Input [1]: [s_store_sk#10]
+Arguments: [s_store_sk#10]
+
+(13) CometBroadcastHashJoin
+Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4]
+Right output [1]: [s_store_sk#10]
+Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight
+
+(14) CometProject
+Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10]
+Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4]
+
+(15) Scan parquet spark_catalog.default.household_demographics
+Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/household_demographics]
+PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)]
+ReadSchema: struct
+
+(16) CometFilter
+Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15]
+Condition : ((((isnotnull(hd_vehicle_count#15) AND ((hd_buy_potential#13 = >10000 ) OR (hd_buy_potential#13 = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN ((cast(hd_dep_count#14 as double) / cast(hd_vehicle_count#15 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#12))
+
+(17) CometProject
+Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15]
+Arguments: [hd_demo_sk#12], [hd_demo_sk#12]
+
+(18) CometBroadcastExchange
+Input [1]: [hd_demo_sk#12]
+Arguments: [hd_demo_sk#12]
+
+(19) CometBroadcastHashJoin
+Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4]
+Right output [1]: [hd_demo_sk#12]
+Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight
+
+(20) CometProject
+Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12]
+Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4]
+
+(21) CometHashAggregate
+Input [2]: [ss_customer_sk#1, ss_ticket_number#4]
+Keys [2]: [ss_ticket_number#4, ss_customer_sk#1]
+Functions [1]: [partial_count(1)]
+
+(22) ColumnarToRow [codegen id : 1]
+Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16]
+
+(23) Exchange
+Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16]
+Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(24) HashAggregate [codegen id : 3]
+Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16]
+Keys [2]: [ss_ticket_number#4, ss_customer_sk#1]
+Functions [1]: [count(1)]
+Aggregate Attributes [1]: [count(1)#17]
+Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18]
+
+(25) Filter [codegen id : 3]
+Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18]
+Condition : ((cnt#18 >= 15) AND (cnt#18 <= 20))
+
+(26) Scan parquet spark_catalog.default.customer
+Output [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_customer_sk)]
+ReadSchema: struct
+
+(27) CometFilter
+Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23]
+Condition : isnotnull(c_customer_sk#19)
+
+(28) ColumnarToRow [codegen id : 2]
+Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23]
+
+(29) BroadcastExchange
+Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2]
+
+(30) BroadcastHashJoin [codegen id : 3]
+Left keys [1]: [ss_customer_sk#1]
+Right keys [1]: [c_customer_sk#19]
+Join type: Inner
+Join condition: None
+
+(31) Project [codegen id : 3]
+Output [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18]
+Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18, c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23]
+
+(32) Exchange
+Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18]
+Arguments: rangepartitioning(c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=3]
+
+(33) Sort [codegen id : 4]
+Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18]
+Arguments: [c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST], true, 0
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6
+BroadcastExchange (38)
++- * ColumnarToRow (37)
+ +- CometProject (36)
+ +- CometFilter (35)
+ +- CometScan parquet spark_catalog.default.date_dim (34)
+
+
+(34) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#7, d_year#8, d_dom#9]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(35) CometFilter
+Input [3]: [d_date_sk#7, d_year#8, d_dom#9]
+Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7))
+
+(36) CometProject
+Input [3]: [d_date_sk#7, d_year#8, d_dom#9]
+Arguments: [d_date_sk#7], [d_date_sk#7]
+
+(37) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#7]
+
+(38) BroadcastExchange
+Input [1]: [d_date_sk#7]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt
new file mode 100644
index 000000000..eefd38343
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt
@@ -0,0 +1,50 @@
+WholeStageCodegen (4)
+ Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag]
+ InputAdapter
+ Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1
+ WholeStageCodegen (3)
+ Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt]
+ BroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ Filter [cnt]
+ HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count]
+ InputAdapter
+ Exchange [ss_ticket_number,ss_customer_sk] #2
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometProject [ss_customer_sk,ss_ticket_number]
+ CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
+ CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_dom,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
+ CometBroadcastExchange #4
+ CometProject [d_date_sk]
+ CometFilter [d_dom,d_year,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
+ CometBroadcastExchange #5
+ CometProject [s_store_sk]
+ CometFilter [s_county,s_store_sk]
+ CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
+ CometBroadcastExchange #6
+ CometProject [hd_demo_sk]
+ CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [c_customer_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt
new file mode 100644
index 000000000..c06c1dd16
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt
@@ -0,0 +1,292 @@
+== Physical Plan ==
+TakeOrderedAndProject (44)
++- * HashAggregate (43)
+ +- Exchange (42)
+ +- * HashAggregate (41)
+ +- * Project (40)
+ +- * BroadcastHashJoin Inner BuildRight (39)
+ :- * Project (34)
+ : +- * BroadcastHashJoin Inner BuildRight (33)
+ : :- * Project (28)
+ : : +- * Filter (27)
+ : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26)
+ : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19)
+ : : : :- * ColumnarToRow (12)
+ : : : : +- CometBroadcastHashJoin (11)
+ : : : : :- CometFilter (2)
+ : : : : : +- CometScan parquet spark_catalog.default.customer (1)
+ : : : : +- CometBroadcastExchange (10)
+ : : : : +- CometProject (9)
+ : : : : +- CometBroadcastHashJoin (8)
+ : : : : :- CometScan parquet spark_catalog.default.store_sales (3)
+ : : : : +- CometBroadcastExchange (7)
+ : : : : +- CometProject (6)
+ : : : : +- CometFilter (5)
+ : : : : +- CometScan parquet spark_catalog.default.date_dim (4)
+ : : : +- BroadcastExchange (18)
+ : : : +- * ColumnarToRow (17)
+ : : : +- CometProject (16)
+ : : : +- CometBroadcastHashJoin (15)
+ : : : :- CometScan parquet spark_catalog.default.web_sales (13)
+ : : : +- ReusedExchange (14)
+ : : +- BroadcastExchange (25)
+ : : +- * ColumnarToRow (24)
+ : : +- CometProject (23)
+ : : +- CometBroadcastHashJoin (22)
+ : : :- CometScan parquet spark_catalog.default.catalog_sales (20)
+ : : +- ReusedExchange (21)
+ : +- BroadcastExchange (32)
+ : +- * ColumnarToRow (31)
+ : +- CometFilter (30)
+ : +- CometScan parquet spark_catalog.default.customer_address (29)
+ +- BroadcastExchange (38)
+ +- * ColumnarToRow (37)
+ +- CometFilter (36)
+ +- CometScan parquet spark_catalog.default.customer_demographics (35)
+
+
+(1) Scan parquet spark_catalog.default.customer
+Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer]
+PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4))
+
+(3) Scan parquet spark_catalog.default.store_sales
+Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)]
+ReadSchema: struct
+
+(4) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#9, d_year#10, d_qoy#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(5) CometFilter
+Input [3]: [d_date_sk#9, d_year#10, d_qoy#11]
+Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9))
+
+(6) CometProject
+Input [3]: [d_date_sk#9, d_year#10, d_qoy#11]
+Arguments: [d_date_sk#9], [d_date_sk#9]
+
+(7) CometBroadcastExchange
+Input [1]: [d_date_sk#9]
+Arguments: [d_date_sk#9]
+
+(8) CometBroadcastHashJoin
+Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7]
+Right output [1]: [d_date_sk#9]
+Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight
+
+(9) CometProject
+Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9]
+Arguments: [ss_customer_sk#6], [ss_customer_sk#6]
+
+(10) CometBroadcastExchange
+Input [1]: [ss_customer_sk#6]
+Arguments: [ss_customer_sk#6]
+
+(11) CometBroadcastHashJoin
+Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+Right output [1]: [ss_customer_sk#6]
+Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight
+
+(12) ColumnarToRow [codegen id : 5]
+Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5]
+
+(13) Scan parquet spark_catalog.default.web_sales
+Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)]
+ReadSchema: struct
+
+(14) ReusedExchange [Reuses operator id: 7]
+Output [1]: [d_date_sk#15]
+
+(15) CometBroadcastHashJoin
+Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13]
+Right output [1]: [d_date_sk#15]
+Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight
+
+(16) CometProject
+Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15]
+Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12]
+
+(17) ColumnarToRow [codegen id : 1]
+Input [1]: [ws_bill_customer_sk#12]
+
+(18) BroadcastExchange
+Input [1]: [ws_bill_customer_sk#12]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1]
+
+(19) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [ws_bill_customer_sk#12]
+Join type: ExistenceJoin(exists#2)
+Join condition: None
+
+(20) Scan parquet spark_catalog.default.catalog_sales
+Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)]
+ReadSchema: struct
+
+(21) ReusedExchange [Reuses operator id: 7]
+Output [1]: [d_date_sk#19]
+
+(22) CometBroadcastHashJoin
+Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17]
+Right output [1]: [d_date_sk#19]
+Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight
+
+(23) CometProject
+Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19]
+Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16]
+
+(24) ColumnarToRow [codegen id : 2]
+Input [1]: [cs_ship_customer_sk#16]
+
+(25) BroadcastExchange
+Input [1]: [cs_ship_customer_sk#16]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2]
+
+(26) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_customer_sk#3]
+Right keys [1]: [cs_ship_customer_sk#16]
+Join type: ExistenceJoin(exists#1)
+Join condition: None
+
+(27) Filter [codegen id : 5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+Condition : (exists#2 OR exists#1)
+
+(28) Project [codegen id : 5]
+Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5]
+Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1]
+
+(29) Scan parquet spark_catalog.default.customer_address
+Output [2]: [ca_address_sk#20, ca_state#21]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_address]
+PushedFilters: [IsNotNull(ca_address_sk)]
+ReadSchema: struct
+
+(30) CometFilter
+Input [2]: [ca_address_sk#20, ca_state#21]
+Condition : isnotnull(ca_address_sk#20)
+
+(31) ColumnarToRow [codegen id : 3]
+Input [2]: [ca_address_sk#20, ca_state#21]
+
+(32) BroadcastExchange
+Input [2]: [ca_address_sk#20, ca_state#21]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+
+(33) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_addr_sk#5]
+Right keys [1]: [ca_address_sk#20]
+Join type: Inner
+Join condition: None
+
+(34) Project [codegen id : 5]
+Output [2]: [c_current_cdemo_sk#4, ca_state#21]
+Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21]
+
+(35) Scan parquet spark_catalog.default.customer_demographics
+Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/customer_demographics]
+PushedFilters: [IsNotNull(cd_demo_sk)]
+ReadSchema: struct
+
+(36) CometFilter
+Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Condition : isnotnull(cd_demo_sk#22)
+
+(37) ColumnarToRow [codegen id : 4]
+Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+
+(38) BroadcastExchange
+Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
+
+(39) BroadcastHashJoin [codegen id : 5]
+Left keys [1]: [c_current_cdemo_sk#4]
+Right keys [1]: [cd_demo_sk#22]
+Join type: Inner
+Join condition: None
+
+(40) Project [codegen id : 5]
+Output [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+
+(41) HashAggregate [codegen id : 5]
+Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_dep_count#25), partial_avg(cd_dep_count#25), partial_min(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_avg(cd_dep_employed_count#26), partial_min(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_avg(cd_dep_college_count#27)]
+Aggregate Attributes [13]: [count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40]
+Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53]
+
+(42) Exchange
+Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53]
+Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+
+(43) HashAggregate [codegen id : 6]
+Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53]
+Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27]
+Functions [10]: [count(1), min(cd_dep_count#25), max(cd_dep_count#25), avg(cd_dep_count#25), min(cd_dep_employed_count#26), max(cd_dep_employed_count#26), avg(cd_dep_employed_count#26), min(cd_dep_college_count#27), max(cd_dep_college_count#27), avg(cd_dep_college_count#27)]
+Aggregate Attributes [10]: [count(1)#54, min(cd_dep_count#25)#55, max(cd_dep_count#25)#56, avg(cd_dep_count#25)#57, min(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, avg(cd_dep_employed_count#26)#60, min(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, avg(cd_dep_college_count#27)#63]
+Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, count(1)#54 AS cnt1#64, min(cd_dep_count#25)#55 AS min(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, min(cd_dep_employed_count#26)#58 AS min(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, min(cd_dep_college_count#27)#61 AS min(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, cd_dep_count#25]
+
+(44) TakeOrderedAndProject
+Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75, cd_dep_count#25]
+Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75]
+
+===== Subqueries =====
+
+Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8
+BroadcastExchange (49)
++- * ColumnarToRow (48)
+ +- CometProject (47)
+ +- CometFilter (46)
+ +- CometScan parquet spark_catalog.default.date_dim (45)
+
+
+(45) Scan parquet spark_catalog.default.date_dim
+Output [3]: [d_date_sk#9, d_year#10, d_qoy#11]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)]
+ReadSchema: struct
+
+(46) CometFilter
+Input [3]: [d_date_sk#9, d_year#10, d_qoy#11]
+Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9))
+
+(47) CometProject
+Input [3]: [d_date_sk#9, d_year#10, d_qoy#11]
+Arguments: [d_date_sk#9], [d_date_sk#9]
+
+(48) ColumnarToRow [codegen id : 1]
+Input [1]: [d_date_sk#9]
+
+(49) BroadcastExchange
+Input [1]: [d_date_sk#9]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
+
+Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8
+
+Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8
+
+
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt
new file mode 100644
index 000000000..efe0b0b4e
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt
@@ -0,0 +1,70 @@
+TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)]
+ WholeStageCodegen (6)
+ HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count]
+ InputAdapter
+ Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1
+ WholeStageCodegen (5)
+ HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count]
+ Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ Project [c_current_cdemo_sk,ca_state]
+ BroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ Project [c_current_cdemo_sk,c_current_addr_sk]
+ Filter [exists,exists]
+ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk]
+ BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
+ ColumnarToRow
+ InputAdapter
+ CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk]
+ CometBroadcastExchange #2
+ CometProject [ss_customer_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
+ SubqueryBroadcast [d_date_sk] #1
+ BroadcastExchange #3
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_qoy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ CometBroadcastExchange #4
+ CometProject [d_date_sk]
+ CometFilter [d_year,d_qoy,d_date_sk]
+ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
+ InputAdapter
+ BroadcastExchange #5
+ WholeStageCodegen (1)
+ ColumnarToRow
+ InputAdapter
+ CometProject [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #4
+ InputAdapter
+ BroadcastExchange #6
+ WholeStageCodegen (2)
+ ColumnarToRow
+ InputAdapter
+ CometProject [cs_ship_customer_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk]
+ ReusedSubquery [d_date_sk] #1
+ ReusedExchange [d_date_sk] #4
+ InputAdapter
+ BroadcastExchange #7
+ WholeStageCodegen (3)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [ca_address_sk]
+ CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
+ InputAdapter
+ BroadcastExchange #8
+ WholeStageCodegen (4)
+ ColumnarToRow
+ InputAdapter
+ CometFilter [cd_demo_sk]
+ CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt
new file mode 100644
index 000000000..33b572ff6
--- /dev/null
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt
@@ -0,0 +1,200 @@
+== Physical Plan ==
+TakeOrderedAndProject (29)
++- * Project (28)
+ +- Window (27)
+ +- * Sort (26)
+ +- Exchange (25)
+ +- * HashAggregate (24)
+ +- Exchange (23)
+ +- * ColumnarToRow (22)
+ +- CometHashAggregate (21)
+ +- CometExpand (20)
+ +- CometProject (19)
+ +- CometBroadcastHashJoin (18)
+ :- CometProject (13)
+ : +- CometBroadcastHashJoin (12)
+ : :- CometProject (8)
+ : : +- CometBroadcastHashJoin (7)
+ : : :- CometFilter (2)
+ : : : +- CometScan parquet spark_catalog.default.store_sales (1)
+ : : +- CometBroadcastExchange (6)
+ : : +- CometProject (5)
+ : : +- CometFilter (4)
+ : : +- CometScan parquet spark_catalog.default.date_dim (3)
+ : +- CometBroadcastExchange (11)
+ : +- CometFilter (10)
+ : +- CometScan parquet spark_catalog.default.item (9)
+ +- CometBroadcastExchange (17)
+ +- CometProject (16)
+ +- CometFilter (15)
+ +- CometScan parquet spark_catalog.default.store (14)
+
+
+(1) Scan parquet spark_catalog.default.store_sales
+Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)]
+PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)]
+ReadSchema: struct
+
+(2) CometFilter
+Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5]
+Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2))
+
+(3) Scan parquet spark_catalog.default.date_dim
+Output [2]: [d_date_sk#7, d_year#8]
+Batched: true
+Location [not included in comparison]/{warehouse_dir}/date_dim]
+PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
+ReadSchema: struct