diff --git a/.github/actions/java-test/action.yaml b/.github/actions/java-test/action.yaml
index e1efd9fce..caf3f6bba 100644
--- a/.github/actions/java-test/action.yaml
+++ b/.github/actions/java-test/action.yaml
@@ -33,7 +33,7 @@ runs:
- name: Run Cargo build
shell: bash
run: |
- cd core
+ cd native
cargo build
- name: Cache Maven dependencies
diff --git a/.github/actions/rust-test/action.yaml b/.github/actions/rust-test/action.yaml
index bf0d0ba15..3aeba0666 100644
--- a/.github/actions/rust-test/action.yaml
+++ b/.github/actions/rust-test/action.yaml
@@ -22,21 +22,27 @@ runs:
- name: Check Cargo fmt
shell: bash
run: |
- cd core
+ cd native
cargo fmt --all -- --check --color=never
- name: Check Cargo clippy
shell: bash
run: |
- cd core
+ cd native
cargo clippy --color=never -- -D warnings
- name: Check compilation
shell: bash
run: |
- cd core
+ cd native
cargo check --benches
+ - name: Check unused dependencies
+ shell: bash
+ run: |
+ cd native
+ cargo install cargo-machete && cargo machete
+
- name: Cache Maven dependencies
uses: actions/cache@v4
with:
@@ -56,5 +62,5 @@ runs:
- name: Run Cargo test
shell: bash
run: |
- cd core
+ cd native
RUST_BACKTRACE=1 cargo test
diff --git a/.github/workflows/benchmark-tpch.yml b/.github/workflows/benchmark-tpch.yml
index f4c547a8f..3d4fbb750 100644
--- a/.github/workflows/benchmark-tpch.yml
+++ b/.github/workflows/benchmark-tpch.yml
@@ -76,8 +76,8 @@ jobs:
with:
name: libcomet-${{ github.run_id }}
path: |
- core/target/release/libcomet.so
- core/target/release/libcomet.dylib
+ native/target/release/libcomet.so
+ native/target/release/libcomet.dylib
retention-days: 1 # remove the artifact after 1 day, only valid for this workflow
overwrite: true
- name: Generate TPC-H (SF=1) table data
@@ -119,7 +119,7 @@ jobs:
uses: actions/download-artifact@v4
with:
name: libcomet-${{ github.run_id }}
- path: core/target/release
+ path: native/target/release
- name: Run TPC-H queries
run: |
SPARK_HOME=`pwd` SPARK_TPCH_DATA=`pwd`/tpch/sf1_parquet ./mvnw -B -Prelease -Dsuites=org.apache.spark.sql.CometTPCHQuerySuite test
diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml
index 023b6a685..de1ad5773 100644
--- a/.github/workflows/benchmark.yml
+++ b/.github/workflows/benchmark.yml
@@ -83,8 +83,8 @@ jobs:
with:
name: libcomet-${{ github.run_id }}
path: |
- core/target/release/libcomet.so
- core/target/release/libcomet.dylib
+ native/target/release/libcomet.so
+ native/target/release/libcomet.dylib
retention-days: 1 # remove the artifact after 1 day, only valid for this workflow
overwrite: true
- name: Build tpcds-kit
@@ -134,7 +134,7 @@ jobs:
uses: actions/download-artifact@v4
with:
name: libcomet-${{ github.run_id }}
- path: core/target/release
+ path: native/target/release
- name: Run TPC-DS queries (Sort merge join)
if: matrix.join == 'sort_merge'
run: |
diff --git a/.gitignore b/.gitignore
index 1c247dd9a..8bdcd51d3 100644
--- a/.gitignore
+++ b/.gitignore
@@ -8,7 +8,7 @@ derby.log
metastore_db/
spark-warehouse/
dependency-reduced-pom.xml
-core/src/execution/generated
+native/core/src/execution/generated
prebuild
.flattened-pom.xml
rat.txt
diff --git a/Makefile b/Makefile
index 573a7f955..a52524511 100644
--- a/Makefile
+++ b/Makefile
@@ -20,11 +20,11 @@
all: core jvm
core:
- cd core && cargo build
+ cd native && cargo build
test-rust:
# We need to compile CometException so that the cargo test can pass
./mvnw compile -pl common -DskipTests $(PROFILES)
- cd core && cargo build && \
+ cd native && cargo build && \
RUST_BACKTRACE=1 cargo test
jvm:
./mvnw clean package -DskipTests $(PROFILES)
@@ -32,24 +32,24 @@ test-jvm: core
SPARK_HOME=`pwd` COMET_CONF_DIR=$(shell pwd)/conf RUST_BACKTRACE=1 ./mvnw verify $(PROFILES)
test: test-rust test-jvm
clean:
- cd core && cargo clean
+ cd native && cargo clean
./mvnw clean $(PROFILES)
rm -rf .dist
bench:
- cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo bench $(filter-out $@,$(MAKECMDGOALS))
+ cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo bench $(filter-out $@,$(MAKECMDGOALS))
format:
- cd core && cargo fmt
+ cd native && cargo fmt
./mvnw compile test-compile scalafix:scalafix -Psemanticdb $(PROFILES)
./mvnw spotless:apply $(PROFILES)
core-amd64:
rustup target add x86_64-apple-darwin
- cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release
+ cd native && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release
mkdir -p common/target/classes/org/apache/comet/darwin/x86_64
- cp core/target/x86_64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/x86_64
- cd core && RUSTFLAGS="-Ctarget-cpu=haswell -Ctarget-feature=-prefer-256-bit" cargo build --release
+ cp native/target/x86_64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/x86_64
+ cd native && RUSTFLAGS="-Ctarget-cpu=haswell -Ctarget-feature=-prefer-256-bit" cargo build --release
mkdir -p common/target/classes/org/apache/comet/linux/amd64
- cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/amd64
+ cp native/target/release/libcomet.so common/target/classes/org/apache/comet/linux/amd64
jar -cf common/target/comet-native-x86_64.jar \
-C common/target/classes/org/apache/comet darwin \
-C common/target/classes/org/apache/comet linux
@@ -57,12 +57,12 @@ core-amd64:
core-arm64:
rustup target add aarch64-apple-darwin
- cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release
+ cd native && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release
mkdir -p common/target/classes/org/apache/comet/darwin/aarch64
- cp core/target/aarch64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/aarch64
- cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release
+ cp native/target/aarch64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/aarch64
+ cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release
mkdir -p common/target/classes/org/apache/comet/linux/aarch64
- cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/aarch64
+ cp native/target/release/libcomet.so common/target/classes/org/apache/comet/linux/aarch64
jar -cf common/target/comet-native-aarch64.jar \
-C common/target/classes/org/apache/comet darwin \
-C common/target/classes/org/apache/comet linux
@@ -70,15 +70,15 @@ core-arm64:
release-linux: clean
rustup target add aarch64-apple-darwin x86_64-apple-darwin
- cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release
- cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release
- cd core && RUSTFLAGS="-Ctarget-cpu=native -Ctarget-feature=-prefer-256-bit" cargo build --release
+ cd native && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release
+ cd native && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release
+ cd native && RUSTFLAGS="-Ctarget-cpu=native -Ctarget-feature=-prefer-256-bit" cargo build --release
./mvnw install -Prelease -DskipTests $(PROFILES)
release:
- cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release
+ cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release
./mvnw install -Prelease -DskipTests $(PROFILES)
release-nogit:
- cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release
+ cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release
./mvnw install -Prelease -DskipTests $(PROFILES) -Dmaven.gitcommitid.skip=true
benchmark-%: clean release
cd spark && COMET_CONF_DIR=$(shell pwd)/conf MAVEN_OPTS='-Xmx20g' ../mvnw exec:java -Dexec.mainClass="$*" -Dexec.classpathScope="test" -Dexec.cleanupDaemonThreads="false" -Dexec.args="$(filter-out $@,$(MAKECMDGOALS))" $(PROFILES)
diff --git a/common/pom.xml b/common/pom.xml
index b59d7b187..b912e8bd5 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -193,14 +193,14 @@ under the License.
${project.basedir}/src/main/resources
- ${project.basedir}/../core/target/x86_64-apple-darwin/release
+ ${project.basedir}/../native/target/x86_64-apple-darwin/release
libcomet.dylib
org/apache/comet/darwin/x86_64
- ${project.basedir}/../core/target/aarch64-apple-darwin/release
+ ${project.basedir}/../native/target/aarch64-apple-darwin/release
libcomet.dylib
diff --git a/common/src/main/scala/org/apache/comet/package.scala b/common/src/main/scala/org/apache/comet/package.scala
index c9aca7538..f44139ba6 100644
--- a/common/src/main/scala/org/apache/comet/package.scala
+++ b/common/src/main/scala/org/apache/comet/package.scala
@@ -21,8 +21,19 @@ package org.apache
import java.util.Properties
+import org.apache.arrow.memory.RootAllocator
+
package object comet {
+ /**
+ * The root allocator for Comet execution. Because Arrow Java memory management is based on
+ * reference counting, exposed arrays increase the reference count of the underlying buffers.
+ * Until the reference count is zero, the memory will not be released. If the consumer side is
+ * finished later than the close of the allocator, the allocator will think the memory is
+ * leaked. To avoid this, we use a single allocator for the whole execution process.
+ */
+ val CometArrowAllocator = new RootAllocator(Long.MaxValue)
+
/**
* Provides access to build information about the Comet libraries. This will be used by the
* benchmarking software to provide the source revision and repository. In addition, the build
diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala
index 595c0a427..89f79c9cd 100644
--- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala
+++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala
@@ -22,18 +22,18 @@ package org.apache.comet.vector
import scala.collection.mutable
import org.apache.arrow.c.{ArrowArray, ArrowImporter, ArrowSchema, CDataDictionaryProvider, Data}
-import org.apache.arrow.memory.RootAllocator
import org.apache.arrow.vector.VectorSchemaRoot
import org.apache.arrow.vector.dictionary.DictionaryProvider
import org.apache.spark.SparkException
import org.apache.spark.sql.comet.util.Utils
import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.comet.CometArrowAllocator
+
class NativeUtil {
import Utils._
- private val allocator = new RootAllocator(Long.MaxValue)
- .newChildAllocator(this.getClass.getSimpleName, 0, Long.MaxValue)
+ private val allocator = CometArrowAllocator
private val dictionaryProvider: CDataDictionaryProvider = new CDataDictionaryProvider
private val importer = new ArrowImporter(allocator)
diff --git a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala
index 4a08f0521..b8106a96e 100644
--- a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala
+++ b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala
@@ -21,20 +21,20 @@ package org.apache.comet.vector
import java.nio.channels.ReadableByteChannel
-import org.apache.arrow.memory.RootAllocator
import org.apache.arrow.vector.VectorSchemaRoot
import org.apache.arrow.vector.ipc.{ArrowStreamReader, ReadChannel}
import org.apache.arrow.vector.ipc.message.MessageChannelReader
import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.comet.CometArrowAllocator
+
/**
* A reader that consumes Arrow data from an input channel, and produces Comet batches.
*/
case class StreamReader(channel: ReadableByteChannel, source: String) extends AutoCloseable {
- private var allocator = new RootAllocator(Long.MaxValue)
- .newChildAllocator(s"${this.getClass.getSimpleName}/$source", 0, Long.MaxValue)
- private val channelReader = new MessageChannelReader(new ReadChannel(channel), allocator)
- private var arrowReader = new ArrowStreamReader(channelReader, allocator)
+ private val channelReader =
+ new MessageChannelReader(new ReadChannel(channel), CometArrowAllocator)
+ private var arrowReader = new ArrowStreamReader(channelReader, CometArrowAllocator)
private var root = arrowReader.getVectorSchemaRoot
def nextBatch(): Option[ColumnarBatch] = {
@@ -53,11 +53,9 @@ case class StreamReader(channel: ReadableByteChannel, source: String) extends Au
if (root != null) {
arrowReader.close()
root.close()
- allocator.close()
arrowReader = null
root = null
- allocator = null
}
}
}
diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff
index 8129c7455..3c88d6a7e 100644
--- a/dev/diffs/3.4.3.diff
+++ b/dev/diffs/3.4.3.diff
@@ -2491,8 +2491,8 @@ index dd55fcfe42c..293e9dc2986 100644
val schema = df.schema
val withoutFilters = df.queryExecution.executedPlan.transform {
case FilterExec(_, child) => child
-+ case CometFilterExec(_, _, _, child, _) => child
-+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child
++ case CometFilterExec(_, _, _, _, child, _) => child
++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child
}
spark.internalCreateDataFrame(withoutFilters.execute(), schema)
diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff
index ebc4cdb8c..cf6754416 100644
--- a/dev/diffs/3.5.1.diff
+++ b/dev/diffs/3.5.1.diff
@@ -2650,8 +2650,8 @@ index dd55fcfe42c..293e9dc2986 100644
val schema = df.schema
val withoutFilters = df.queryExecution.executedPlan.transform {
case FilterExec(_, child) => child
-+ case CometFilterExec(_, _, _, child, _) => child
-+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child
++ case CometFilterExec(_, _, _, _, child, _) => child
++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child
}
spark.internalCreateDataFrame(withoutFilters.execute(), schema)
diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff
index 4031015df..abd6e9a04 100644
--- a/dev/diffs/4.0.0-preview1.diff
+++ b/dev/diffs/4.0.0-preview1.diff
@@ -415,7 +415,7 @@ index 16a493b5290..3f0b70e2d59 100644
assert(exchanges.size == 2)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
-index 2c24cc7d570..50a2ce86117 100644
+index 2c24cc7d570..d46dc5e138a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala
@@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen
@@ -442,7 +442,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("partition pruning in broadcast hash joins with aliases") {
+ test("partition pruning in broadcast hash joins with aliases",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
Given("alias with simple join condition, using attribute names only")
withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") {
val df = sql(
@@ -452,7 +452,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("partition pruning in broadcast hash joins") {
+ test("partition pruning in broadcast hash joins",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
Given("disable broadcast pruning and disable subquery duplication")
withSQLConf(
SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true",
@@ -462,7 +462,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("different broadcast subqueries with identical children") {
+ test("different broadcast subqueries with identical children",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") {
withTable("fact", "dim") {
spark.range(100).select(
@@ -492,7 +492,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("SPARK-32817: DPP throws error when the broadcast side is empty") {
+ test("SPARK-32817: DPP throws error when the broadcast side is empty",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
withSQLConf(
SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true",
SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true",
@@ -502,7 +502,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning") {
+ test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") {
val df = sql(
"""
@@ -521,7 +521,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") {
+ test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
withTable("duplicate_keys") {
withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") {
Seq[(Int, String)]((1, "NL"), (1, "NL"), (3, "US"), (3, "US"), (3, "US"))
@@ -531,7 +531,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty") {
+ test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") {
val df = sql(
"""
@@ -541,7 +541,7 @@ index 2c24cc7d570..50a2ce86117 100644
- test("SPARK-39217: Makes DPP support the pruning side has Union") {
+ test("SPARK-39217: Makes DPP support the pruning side has Union",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") {
val df = sql(
"""
@@ -1159,7 +1159,7 @@ index 15de4c5cc5b..6a85dfb6883 100644
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
-index b5bac8079c4..544c1ddc697 100644
+index b5bac8079c4..a3731888e12 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
@@ -17,7 +17,8 @@
@@ -1190,7 +1190,7 @@ index b5bac8079c4..544c1ddc697 100644
- test("join with ordering requirement") {
+ test("join with ordering requirement",
-+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) {
++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) {
val query = "select * from (select key, a, c, b from testView) as t1 join " +
"(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50"
assertProjectExec(query, 2, 2)
@@ -2590,7 +2590,7 @@ index af07aceaed1..ed0b5e6d9be 100644
val tblTargetName = "tbl_target"
val tblSourceQualified = s"default.$tblSourceName"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
-index 5fbf379644f..32711763ec1 100644
+index 5fbf379644f..6153046a787 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
@@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest
@@ -2651,8 +2651,8 @@ index 5fbf379644f..32711763ec1 100644
val schema = df.schema
val withoutFilters = df.queryExecution.executedPlan.transform {
case FilterExec(_, child) => child
-+ case CometFilterExec(_, _, _, child, _) => child
-+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child
++ case CometFilterExec(_, _, _, _, child, _) => child
++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child
}
spark.internalCreateDataFrame(withoutFilters.execute(), schema)
diff --git a/core/Cargo.lock b/native/Cargo.lock
similarity index 88%
rename from core/Cargo.lock
rename to native/Cargo.lock
index 71fe6eb2e..c0f22fa1a 100644
--- a/core/Cargo.lock
+++ b/native/Cargo.lock
@@ -114,8 +114,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711"
[[package]]
name = "arrow"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "6127ea5e585a12ec9f742232442828ebaf264dfa5eefdd71282376c599562b77"
dependencies = [
"arrow-arith",
"arrow-array",
@@ -134,8 +135,9 @@ dependencies = [
[[package]]
name = "arrow-arith"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "7add7f39210b7d726e2a8efc0083e7bf06e8f2d15bdb4896b564dce4410fbf5d"
dependencies = [
"arrow-array",
"arrow-buffer",
@@ -148,8 +150,9 @@ dependencies = [
[[package]]
name = "arrow-array"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "81c16ec702d3898c2f5cfdc148443c6cd7dbe5bac28399859eb0a3d38f072827"
dependencies = [
"ahash",
"arrow-buffer",
@@ -164,8 +167,9 @@ dependencies = [
[[package]]
name = "arrow-buffer"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "cae6970bab043c4fbc10aee1660ceb5b306d0c42c8cc5f6ae564efcd9759b663"
dependencies = [
"bytes",
"half",
@@ -174,8 +178,9 @@ dependencies = [
[[package]]
name = "arrow-cast"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "1c7ef44f26ef4f8edc392a048324ed5d757ad09135eff6d5509e6450d39e0398"
dependencies = [
"arrow-array",
"arrow-buffer",
@@ -194,8 +199,9 @@ dependencies = [
[[package]]
name = "arrow-csv"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "5f843490bd258c5182b66e888161bb6f198f49f3792f7c7f98198b924ae0f564"
dependencies = [
"arrow-array",
"arrow-buffer",
@@ -212,8 +218,9 @@ dependencies = [
[[package]]
name = "arrow-data"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "a769666ffac256dd301006faca1ca553d0ae7cffcf4cd07095f73f95eb226514"
dependencies = [
"arrow-buffer",
"arrow-schema",
@@ -223,8 +230,9 @@ dependencies = [
[[package]]
name = "arrow-ipc"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "dbf9c3fb57390a1af0b7bb3b5558c1ee1f63905f3eccf49ae7676a8d1e6e5a72"
dependencies = [
"arrow-array",
"arrow-buffer",
@@ -237,8 +245,9 @@ dependencies = [
[[package]]
name = "arrow-json"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "654e7f3724176b66ddfacba31af397c48e106fbe4d281c8144e7d237df5acfd7"
dependencies = [
"arrow-array",
"arrow-buffer",
@@ -256,8 +265,9 @@ dependencies = [
[[package]]
name = "arrow-ord"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e8008370e624e8e3c68174faaf793540287106cfda8ad1da862fdc53d8e096b4"
dependencies = [
"arrow-array",
"arrow-buffer",
@@ -270,8 +280,9 @@ dependencies = [
[[package]]
name = "arrow-row"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "ca5e3a6b7fda8d9fe03f3b18a2d946354ea7f3c8e4076dbdb502ad50d9d44824"
dependencies = [
"ahash",
"arrow-array",
@@ -284,16 +295,18 @@ dependencies = [
[[package]]
name = "arrow-schema"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "dab1c12b40e29d9f3b699e0203c2a73ba558444c05e388a4377208f8f9c97eee"
dependencies = [
- "bitflags 2.5.0",
+ "bitflags 2.6.0",
]
[[package]]
name = "arrow-select"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "e80159088ffe8c48965cb9b1a7c968b2729f29f37363df7eca177fc3281fe7c3"
dependencies = [
"ahash",
"arrow-array",
@@ -305,8 +318,9 @@ dependencies = [
[[package]]
name = "arrow-string"
-version = "52.0.0"
-source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10"
+version = "52.1.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "0fd04a6ea7de183648edbcb7a6dd925bbd04c210895f6384c780e27a9b54afcd"
dependencies = [
"arrow-array",
"arrow-buffer",
@@ -327,13 +341,13 @@ checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131"
[[package]]
name = "async-trait"
-version = "0.1.80"
+version = "0.1.81"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca"
+checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
@@ -353,9 +367,9 @@ checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0"
[[package]]
name = "backtrace"
-version = "0.3.72"
+version = "0.3.73"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "17c6a35df3749d2e8bb1b7b21a976d82b15548788d2735b9d82f329268f71a11"
+checksum = "5cc23269a4f8976d0a4d2e7109211a419fe30e8d88d677cd60b6bc79c5732e0a"
dependencies = [
"addr2line",
"cc",
@@ -380,9 +394,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a"
[[package]]
name = "bitflags"
-version = "2.5.0"
+version = "2.6.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1"
+checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de"
[[package]]
name = "blake2"
@@ -444,9 +458,9 @@ checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c"
[[package]]
name = "bytemuck"
-version = "1.16.0"
+version = "1.16.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "78834c15cb5d5efe3452d58b1e8ba890dd62d21907f867f383358198e56ebca5"
+checksum = "b236fc92302c97ed75b38da1f4917b5cdda4984745740f153a5d3059e48d725e"
[[package]]
name = "byteorder"
@@ -468,9 +482,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5"
[[package]]
name = "cc"
-version = "1.0.99"
+version = "1.0.106"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "96c51067fd44124faa7f870b4b1c969379ad32b2ba805aa959430ceaa384f695"
+checksum = "066fce287b1d4eafef758e89e09d724a24808a9196fe9756b8ca90e86d0719a2"
dependencies = [
"jobserver",
"libc",
@@ -498,7 +512,7 @@ dependencies = [
"android-tzdata",
"iana-time-zone",
"num-traits",
- "windows-targets 0.52.5",
+ "windows-targets 0.52.6",
]
[[package]]
@@ -574,18 +588,18 @@ dependencies = [
[[package]]
name = "clap"
-version = "4.5.6"
+version = "4.5.8"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a9689a29b593160de5bc4aacab7b5d54fb52231de70122626c178e6a368994c7"
+checksum = "84b3edb18336f4df585bc9aa31dd99c036dfa5dc5e9a2939a722a188f3a8970d"
dependencies = [
"clap_builder",
]
[[package]]
name = "clap_builder"
-version = "4.5.6"
+version = "4.5.8"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "2e5387378c84f6faa26890ebf9f0a92989f8873d4d380467bcd0d8d8620424df"
+checksum = "c1c09dd5ada6c6c78075d6fd0da3f90d8080651e2d6cc8eb2f1aaa4034ced708"
dependencies = [
"anstyle",
"clap_lex",
@@ -790,8 +804,8 @@ dependencies = [
[[package]]
name = "datafusion"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"ahash",
"arrow",
@@ -843,26 +857,23 @@ dependencies = [
"arrow-buffer",
"arrow-data",
"arrow-schema",
- "arrow-string",
"assertables",
"async-trait",
"brotli",
- "byteorder",
"bytes",
"chrono",
"chrono-tz 0.8.6",
"crc32fast",
"criterion",
"datafusion",
+ "datafusion-comet-spark-expr",
"datafusion-common",
"datafusion-expr",
- "datafusion-functions",
"datafusion-physical-expr",
"datafusion-physical-expr-common",
"flate2",
"futures",
"half",
- "hashbrown",
"hex",
"itertools 0.11.0",
"jni",
@@ -873,9 +884,7 @@ dependencies = [
"mimalloc",
"num",
"once_cell",
- "parking_lot",
"parquet",
- "parquet-format",
"paste",
"pprof",
"prost 0.12.6",
@@ -887,18 +896,27 @@ dependencies = [
"snap",
"tempfile",
"thiserror",
- "thrift 0.17.0",
"tokio",
- "tokio-stream",
"twox-hash",
"unicode-segmentation",
"zstd",
]
+[[package]]
+name = "datafusion-comet-spark-expr"
+version = "0.1.0"
+dependencies = [
+ "arrow",
+ "arrow-schema",
+ "datafusion",
+ "datafusion-common",
+ "datafusion-functions",
+]
+
[[package]]
name = "datafusion-common"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"ahash",
"arrow",
@@ -917,16 +935,16 @@ dependencies = [
[[package]]
name = "datafusion-common-runtime"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"tokio",
]
[[package]]
name = "datafusion-execution"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"arrow",
"chrono",
@@ -945,8 +963,8 @@ dependencies = [
[[package]]
name = "datafusion-expr"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"ahash",
"arrow",
@@ -963,8 +981,8 @@ dependencies = [
[[package]]
name = "datafusion-functions"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"arrow",
"base64",
@@ -974,7 +992,6 @@ dependencies = [
"datafusion-common",
"datafusion-execution",
"datafusion-expr",
- "datafusion-physical-expr",
"hashbrown",
"hex",
"itertools 0.12.1",
@@ -989,8 +1006,8 @@ dependencies = [
[[package]]
name = "datafusion-functions-aggregate"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"ahash",
"arrow",
@@ -1006,8 +1023,8 @@ dependencies = [
[[package]]
name = "datafusion-optimizer"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"arrow",
"async-trait",
@@ -1019,13 +1036,14 @@ dependencies = [
"indexmap",
"itertools 0.12.1",
"log",
+ "paste",
"regex-syntax",
]
[[package]]
name = "datafusion-physical-expr"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"ahash",
"arrow",
@@ -1039,7 +1057,6 @@ dependencies = [
"datafusion-common",
"datafusion-execution",
"datafusion-expr",
- "datafusion-functions-aggregate",
"datafusion-physical-expr-common",
"half",
"hashbrown",
@@ -1054,19 +1071,21 @@ dependencies = [
[[package]]
name = "datafusion-physical-expr-common"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
+ "ahash",
"arrow",
"datafusion-common",
"datafusion-expr",
+ "hashbrown",
"rand",
]
[[package]]
name = "datafusion-physical-plan"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"ahash",
"arrow",
@@ -1098,8 +1117,8 @@ dependencies = [
[[package]]
name = "datafusion-sql"
-version = "39.0.0"
-source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569"
+version = "40.0.0"
+source = "git+https://github.com/apache/datafusion.git?rev=40.0.0-rc1#4cae81363e29f011c6602a7a7a54e1aaee841046"
dependencies = [
"arrow",
"arrow-array",
@@ -1157,9 +1176,9 @@ checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10"
[[package]]
name = "either"
-version = "1.12.0"
+version = "1.13.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3dca9240753cf90908d7e4aac30f630662b02aebaa1b58a3cadabdb23385b58b"
+checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0"
[[package]]
name = "equivalent"
@@ -1292,7 +1311,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
@@ -1500,12 +1519,6 @@ dependencies = [
"web-sys",
]
-[[package]]
-name = "integer-encoding"
-version = "1.1.7"
-source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f"
-
[[package]]
name = "integer-encoding"
version = "3.0.4"
@@ -1610,9 +1623,9 @@ dependencies = [
[[package]]
name = "lazy_static"
-version = "1.4.0"
+version = "1.5.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646"
+checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe"
[[package]]
name = "lexical-core"
@@ -1702,9 +1715,9 @@ checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058"
[[package]]
name = "libmimalloc-sys"
-version = "0.1.38"
+version = "0.1.39"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "0e7bb23d733dfcc8af652a78b7bf232f0e967710d044732185e561e47c0336b6"
+checksum = "23aa6811d3bd4deb8a84dde645f943476d13b248d818edcf8ce0b2f37f036b44"
dependencies = [
"cc",
"libc",
@@ -1728,9 +1741,9 @@ dependencies = [
[[package]]
name = "log"
-version = "0.4.21"
+version = "0.4.22"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c"
+checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24"
dependencies = [
"serde",
]
@@ -1771,9 +1784,9 @@ dependencies = [
[[package]]
name = "lz4"
-version = "1.24.0"
+version = "1.25.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1"
+checksum = "d6eab492fe7f8651add23237ea56dbf11b3c4ff762ab83d40a47f11433421f91"
dependencies = [
"libc",
"lz4-sys",
@@ -1781,9 +1794,9 @@ dependencies = [
[[package]]
name = "lz4-sys"
-version = "1.9.4"
+version = "1.9.5"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900"
+checksum = "e9764018d143cc854c9f17f0b907de70f14393b1f502da6375dce70f00514eb3"
dependencies = [
"cc",
"libc",
@@ -1810,9 +1823,9 @@ dependencies = [
[[package]]
name = "memchr"
-version = "2.7.2"
+version = "2.7.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d"
+checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3"
[[package]]
name = "memmap2"
@@ -1825,18 +1838,18 @@ dependencies = [
[[package]]
name = "mimalloc"
-version = "0.1.42"
+version = "0.1.43"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "e9186d86b79b52f4a77af65604b51225e8db1d6ee7e3f41aec1e40829c71a176"
+checksum = "68914350ae34959d83f732418d51e2427a794055d0b9529f48259ac07af65633"
dependencies = [
"libmimalloc-sys",
]
[[package]]
name = "miniz_oxide"
-version = "0.7.3"
+version = "0.7.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "87dfd01fe195c66b572b37921ad8803d010623c0aca821bea2302239d155cdae"
+checksum = "b8a240ddb74feaf34a79a7add65a741f3167852fba007066dcac1ca548d89c08"
dependencies = [
"adler",
]
@@ -1874,9 +1887,9 @@ dependencies = [
[[package]]
name = "num-bigint"
-version = "0.4.5"
+version = "0.4.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c165a9ab64cf766f73521c0dd2cfdff64f488b8f0b3e621face3462d3db536d7"
+checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9"
dependencies = [
"num-integer",
"num-traits",
@@ -1954,9 +1967,9 @@ dependencies = [
[[package]]
name = "object"
-version = "0.35.0"
+version = "0.36.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "b8ec7ab813848ba4522158d5517a6093db1ded27575b070f4177b8d12b41db5e"
+checksum = "081b846d1d56ddfc18fdf1a922e4f6e07a11768ea1b92dec44e42b72712ccfce"
dependencies = [
"memchr",
]
@@ -1990,18 +2003,9 @@ checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92"
[[package]]
name = "oorandom"
-version = "11.1.3"
+version = "11.1.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575"
-
-[[package]]
-name = "ordered-float"
-version = "1.1.1"
-source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7"
-dependencies = [
- "num-traits",
-]
+checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9"
[[package]]
name = "ordered-float"
@@ -2032,7 +2036,7 @@ dependencies = [
"libc",
"redox_syscall",
"smallvec",
- "windows-targets 0.52.5",
+ "windows-targets 0.52.6",
]
[[package]]
@@ -2049,19 +2053,10 @@ dependencies = [
"num-bigint",
"paste",
"seq-macro",
- "thrift 0.17.0",
+ "thrift",
"twox-hash",
]
-[[package]]
-name = "parquet-format"
-version = "4.0.0"
-source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5"
-dependencies = [
- "thrift 0.13.0",
-]
-
[[package]]
name = "parse-zoneinfo"
version = "0.3.1"
@@ -2206,9 +2201,9 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de"
[[package]]
name = "proc-macro2"
-version = "1.0.85"
+version = "1.0.86"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23"
+checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77"
dependencies = [
"unicode-ident",
]
@@ -2276,7 +2271,7 @@ dependencies = [
"itertools 0.12.1",
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
@@ -2359,18 +2354,18 @@ dependencies = [
[[package]]
name = "redox_syscall"
-version = "0.5.1"
+version = "0.5.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "469052894dcb553421e483e4209ee581a45100d31b4018de03e5a7ad86374a7e"
+checksum = "c82cf8cff14456045f55ec4241383baeff27af886adb72ffb2162f99911de0fd"
dependencies = [
- "bitflags 2.5.0",
+ "bitflags 2.6.0",
]
[[package]]
name = "regex"
-version = "1.10.4"
+version = "1.10.5"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c117dbdfde9c8308975b6a18d71f3f385c89461f7b3fb054288ecf2a2058ba4c"
+checksum = "b91213439dad192326a0d7c6ee3955910425f441d7038e0d6933b0aec5c4517f"
dependencies = [
"aho-corasick",
"memchr",
@@ -2380,9 +2375,9 @@ dependencies = [
[[package]]
name = "regex-automata"
-version = "0.4.6"
+version = "0.4.7"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea"
+checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df"
dependencies = [
"aho-corasick",
"memchr",
@@ -2391,15 +2386,15 @@ dependencies = [
[[package]]
name = "regex-syntax"
-version = "0.8.3"
+version = "0.8.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "adad44e29e4c806119491a7f06f03de4d1af22c3a680dd47f1e6e179439d1f56"
+checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b"
[[package]]
name = "rgb"
-version = "0.8.37"
+version = "0.8.44"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "05aaa8004b64fd573fc9d002f4e632d51ad4f026c2b5ba95fcb6c2f32c2c47d8"
+checksum = "1aee83dc281d5a3200d37b299acd13b81066ea126a7f16f0eae70fc9aed241d9"
dependencies = [
"bytemuck",
]
@@ -2425,7 +2420,7 @@ version = "0.38.34"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f"
dependencies = [
- "bitflags 2.5.0",
+ "bitflags 2.6.0",
"errno",
"libc",
"linux-raw-sys",
@@ -2473,9 +2468,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4"
[[package]]
name = "serde"
-version = "1.0.203"
+version = "1.0.204"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094"
+checksum = "bc76f558e0cbb2a839d37354c575f1dc3fdc6546b5be373ba43d95f231bf7c12"
dependencies = [
"serde_derive",
]
@@ -2486,26 +2481,26 @@ version = "0.7.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c"
dependencies = [
- "ordered-float 2.10.1",
+ "ordered-float",
"serde",
]
[[package]]
name = "serde_derive"
-version = "1.0.203"
+version = "1.0.204"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba"
+checksum = "e0cd7e117be63d3c3678776753929474f3b04a43a080c744d6b0ae2a8c28e222"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
name = "serde_json"
-version = "1.0.117"
+version = "1.0.120"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "455182ea6142b14f93f4bc5320a2b31c1f266b66a4a5c858b013302a5d8cbfc3"
+checksum = "4e0d21c9a8cae1235ad58a00c11cb40d4b1e5c784f1ef2c537876ed6ffd8b7c5"
dependencies = [
"itoa",
"ryu",
@@ -2609,7 +2604,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
@@ -2632,9 +2627,9 @@ checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb"
[[package]]
name = "strum"
-version = "0.26.2"
+version = "0.26.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29"
+checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06"
dependencies = [
"strum_macros",
]
@@ -2649,14 +2644,14 @@ dependencies = [
"proc-macro2",
"quote",
"rustversion",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
name = "subtle"
-version = "2.5.0"
+version = "2.6.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc"
+checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292"
[[package]]
name = "symbolic-common"
@@ -2694,9 +2689,9 @@ dependencies = [
[[package]]
name = "syn"
-version = "2.0.66"
+version = "2.0.70"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c42f3f41a2de00b01c0aaad383c5a45241efc8b2d1eda5661812fda5f3cdcff5"
+checksum = "2f0209b68b3613b093e0ec905354eccaedcfe83b8cb37cbdeae64026c3064c16"
dependencies = [
"proc-macro2",
"quote",
@@ -2732,7 +2727,7 @@ checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
@@ -2745,28 +2740,6 @@ dependencies = [
"winapi",
]
-[[package]]
-name = "threadpool"
-version = "1.8.1"
-source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa"
-dependencies = [
- "num_cpus",
-]
-
-[[package]]
-name = "thrift"
-version = "0.13.0"
-source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b"
-dependencies = [
- "byteorder",
- "integer-encoding 1.1.7",
- "log",
- "ordered-float 1.1.1",
- "threadpool",
-]
-
[[package]]
name = "thrift"
version = "0.17.0"
@@ -2774,10 +2747,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09"
dependencies = [
"byteorder",
- "integer-encoding 3.0.4",
- "log",
- "ordered-float 2.10.1",
- "threadpool",
+ "integer-encoding",
+ "ordered-float",
]
[[package]]
@@ -2801,9 +2772,9 @@ dependencies = [
[[package]]
name = "tinyvec"
-version = "1.6.0"
+version = "1.8.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50"
+checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938"
dependencies = [
"tinyvec_macros",
]
@@ -2835,18 +2806,7 @@ checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
-]
-
-[[package]]
-name = "tokio-stream"
-version = "0.1.15"
-source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af"
-dependencies = [
- "futures-core",
- "pin-project-lite",
- "tokio",
+ "syn 2.0.70",
]
[[package]]
@@ -2868,7 +2828,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
@@ -2956,9 +2916,9 @@ checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861"
[[package]]
name = "url"
-version = "2.5.0"
+version = "2.5.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633"
+checksum = "22784dbdf76fdde8af1aeda5622b546b422b6fc585325248a2bf9f5e41e94d6c"
dependencies = [
"form_urlencoded",
"idna",
@@ -2967,9 +2927,9 @@ dependencies = [
[[package]]
name = "uuid"
-version = "1.8.0"
+version = "1.9.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a183cf7feeba97b4dd1c0d46788634f6221d87fa961b305bed08c851829efcc0"
+checksum = "5de17fd2f7da591098415cff336e12965a28061ddace43b59cb3c430179c9439"
dependencies = [
"getrandom",
]
@@ -3017,7 +2977,7 @@ dependencies = [
"once_cell",
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
"wasm-bindgen-shared",
]
@@ -3039,7 +2999,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
"wasm-bindgen-backend",
"wasm-bindgen-shared",
]
@@ -3109,7 +3069,7 @@ version = "0.52.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9"
dependencies = [
- "windows-targets 0.52.5",
+ "windows-targets 0.52.6",
]
[[package]]
@@ -3127,7 +3087,7 @@ version = "0.52.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d"
dependencies = [
- "windows-targets 0.52.5",
+ "windows-targets 0.52.6",
]
[[package]]
@@ -3147,18 +3107,18 @@ dependencies = [
[[package]]
name = "windows-targets"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb"
+checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973"
dependencies = [
- "windows_aarch64_gnullvm 0.52.5",
- "windows_aarch64_msvc 0.52.5",
- "windows_i686_gnu 0.52.5",
+ "windows_aarch64_gnullvm 0.52.6",
+ "windows_aarch64_msvc 0.52.6",
+ "windows_i686_gnu 0.52.6",
"windows_i686_gnullvm",
- "windows_i686_msvc 0.52.5",
- "windows_x86_64_gnu 0.52.5",
- "windows_x86_64_gnullvm 0.52.5",
- "windows_x86_64_msvc 0.52.5",
+ "windows_i686_msvc 0.52.6",
+ "windows_x86_64_gnu 0.52.6",
+ "windows_x86_64_gnullvm 0.52.6",
+ "windows_x86_64_msvc 0.52.6",
]
[[package]]
@@ -3169,9 +3129,9 @@ checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8"
[[package]]
name = "windows_aarch64_gnullvm"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263"
+checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3"
[[package]]
name = "windows_aarch64_msvc"
@@ -3181,9 +3141,9 @@ checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43"
[[package]]
name = "windows_aarch64_msvc"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6"
+checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469"
[[package]]
name = "windows_i686_gnu"
@@ -3193,15 +3153,15 @@ checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f"
[[package]]
name = "windows_i686_gnu"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670"
+checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b"
[[package]]
name = "windows_i686_gnullvm"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9"
+checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66"
[[package]]
name = "windows_i686_msvc"
@@ -3211,9 +3171,9 @@ checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060"
[[package]]
name = "windows_i686_msvc"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf"
+checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66"
[[package]]
name = "windows_x86_64_gnu"
@@ -3223,9 +3183,9 @@ checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36"
[[package]]
name = "windows_x86_64_gnu"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9"
+checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78"
[[package]]
name = "windows_x86_64_gnullvm"
@@ -3235,9 +3195,9 @@ checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3"
[[package]]
name = "windows_x86_64_gnullvm"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596"
+checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d"
[[package]]
name = "windows_x86_64_msvc"
@@ -3247,28 +3207,28 @@ checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0"
[[package]]
name = "windows_x86_64_msvc"
-version = "0.52.5"
+version = "0.52.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0"
+checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec"
[[package]]
name = "zerocopy"
-version = "0.7.34"
+version = "0.7.35"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "ae87e3fcd617500e5d106f0380cf7b77f3c6092aae37191433159dda23cfb087"
+checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0"
dependencies = [
"zerocopy-derive",
]
[[package]]
name = "zerocopy-derive"
-version = "0.7.34"
+version = "0.7.35"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "15e934569e47891f7d9411f1a451d947a60e000ab3bd24fbb970f000387d1b3b"
+checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e"
dependencies = [
"proc-macro2",
"quote",
- "syn 2.0.66",
+ "syn 2.0.70",
]
[[package]]
@@ -3292,9 +3252,9 @@ dependencies = [
[[package]]
name = "zstd-sys"
-version = "2.0.10+zstd.1.5.6"
+version = "2.0.12+zstd.1.5.6"
source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa"
+checksum = "0a4e40c320c3cb459d9a9ff6de98cff88f4751ee9275d140e2be94a2b74e4c13"
dependencies = [
"cc",
"pkg-config",
diff --git a/native/Cargo.toml b/native/Cargo.toml
new file mode 100644
index 000000000..13860fbdf
--- /dev/null
+++ b/native/Cargo.toml
@@ -0,0 +1,54 @@
+# 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.
+
+[workspace]
+members = ["core", "spark-expr"]
+resolver = "2"
+
+[workspace.package]
+version = "0.1.0"
+homepage = "https://datafusion.apache.org/comet"
+repository = "https://github.com/apache/datafusion-comet"
+authors = ["Apache DataFusion "]
+description = "Apache DataFusion Comet: High performance accelerator for Apache Spark"
+readme = "README.md"
+license = "Apache-2.0"
+edition = "2021"
+
+# Comet uses the same minimum Rust version as DataFusion
+rust-version = "1.75"
+
+[workspace.dependencies]
+arrow = { version = "52.1.0", features = ["prettyprint", "ffi", "chrono-tz"] }
+arrow-array = { version = "52.1.0" }
+arrow-buffer = { version = "52.1.0" }
+arrow-data = { version = "52.1.0" }
+arrow-schema = { version = "52.1.0" }
+parquet = { version = "52.1.0", default-features = false, features = ["experimental"] }
+datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1" }
+datafusion = { default-features = false, git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["unicode_expressions", "crypto_expressions"] }
+datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", features = ["crypto_expressions"] }
+datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false }
+datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false }
+datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0-rc1", default-features = false }
+
+[profile.release]
+debug = true
+overflow-checks = false
+lto = "thin"
+codegen-units = 1
+strip = "debuginfo"
diff --git a/core/Cargo.toml b/native/core/Cargo.toml
similarity index 64%
rename from core/Cargo.toml
rename to native/core/Cargo.toml
index 2c450aa39..6432118d6 100644
--- a/core/Cargo.toml
+++ b/native/core/Cargo.toml
@@ -31,20 +31,20 @@ include = [
"Cargo.toml",
]
+# this crate is used in the Spark plugin and does not contain public Rust APIs so we do not publish this crate
+publish = false
+
[dependencies]
-parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate
-arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", features = ["prettyprint", "ffi", "chrono-tz"] }
-arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" }
-arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" }
-arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" }
-arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" }
-arrow-string = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" }
+arrow = { workspace = true }
+arrow-array = { workspace = true }
+arrow-buffer = { workspace = true }
+arrow-data = { workspace = true }
+arrow-schema = { workspace = true }
parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] }
half = { version = "2.4.1", default-features = false }
futures = "0.3.28"
mimalloc = { version = "*", default-features = false, optional = true }
tokio = { version = "1", features = ["rt-multi-thread"] }
-tokio-stream = { version = "0.1.8" }
async-trait = "0.1"
log = "0.4"
log4rs = "1.2.0"
@@ -52,9 +52,7 @@ thiserror = "1"
serde = { version = "1", features = ["derive"] }
lazy_static = "1.4.0"
prost = "0.12.1"
-thrift = "0.17"
jni = "0.21"
-byteorder = "1.4.3"
snap = "1.1"
brotli = "3.3"
flate2 = "1.0"
@@ -63,25 +61,23 @@ zstd = "0.11"
rand = "0.8"
num = "0.4"
bytes = "1.5.0"
-hashbrown = { version = "0.14", features = ["raw"] }
-parking_lot = "0.12"
tempfile = "3.8.0"
ahash = { version = "0.8", default-features = false }
itertools = "0.11.0"
chrono = { version = "0.4", default-features = false, features = ["clock"] }
chrono-tz = { version = "0.8" }
paste = "1.0.14"
-datafusion-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1" }
-datafusion = { default-features = false, git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["unicode_expressions", "crypto_expressions"] }
-datafusion-functions = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["crypto_expressions"] }
-datafusion-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false }
-datafusion-physical-expr-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false }
-datafusion-physical-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false }
+datafusion-common = { workspace = true }
+datafusion = { workspace = true }
+datafusion-expr = { workspace = true }
+datafusion-physical-expr-common = { workspace = true }
+datafusion-physical-expr = { workspace = true }
unicode-segmentation = "^1.10.1"
once_cell = "1.18.0"
regex = "1.9.6"
crc32fast = "1.3.2"
simd-adler32 = "0.3.7"
+datafusion-comet-spark-expr = { path = "../spark-expr", version = "0.1.0" }
[build-dependencies]
prost-build = "0.9.0"
@@ -99,13 +95,6 @@ twox-hash = "1.6.3"
default = []
nightly = []
-[profile.release]
-debug = true
-overflow-checks = false
-lto = "thin"
-codegen-units = 1
-strip = "debuginfo"
-
[lib]
name = "comet"
# "rlib" is for benchmarking with criterion.
diff --git a/core/benches/bit_util.rs b/native/core/benches/bit_util.rs
similarity index 100%
rename from core/benches/bit_util.rs
rename to native/core/benches/bit_util.rs
diff --git a/core/benches/cast_from_string.rs b/native/core/benches/cast_from_string.rs
similarity index 100%
rename from core/benches/cast_from_string.rs
rename to native/core/benches/cast_from_string.rs
diff --git a/core/benches/cast_numeric.rs b/native/core/benches/cast_numeric.rs
similarity index 100%
rename from core/benches/cast_numeric.rs
rename to native/core/benches/cast_numeric.rs
diff --git a/core/benches/common.rs b/native/core/benches/common.rs
similarity index 100%
rename from core/benches/common.rs
rename to native/core/benches/common.rs
diff --git a/core/benches/hash.rs b/native/core/benches/hash.rs
similarity index 100%
rename from core/benches/hash.rs
rename to native/core/benches/hash.rs
diff --git a/core/benches/parquet_decode.rs b/native/core/benches/parquet_decode.rs
similarity index 100%
rename from core/benches/parquet_decode.rs
rename to native/core/benches/parquet_decode.rs
diff --git a/core/benches/parquet_read.rs b/native/core/benches/parquet_read.rs
similarity index 100%
rename from core/benches/parquet_read.rs
rename to native/core/benches/parquet_read.rs
diff --git a/core/benches/perf.rs b/native/core/benches/perf.rs
similarity index 100%
rename from core/benches/perf.rs
rename to native/core/benches/perf.rs
diff --git a/core/benches/row_columnar.rs b/native/core/benches/row_columnar.rs
similarity index 100%
rename from core/benches/row_columnar.rs
rename to native/core/benches/row_columnar.rs
diff --git a/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs
similarity index 100%
rename from core/benches/shuffle_writer.rs
rename to native/core/benches/shuffle_writer.rs
diff --git a/core/build.rs b/native/core/build.rs
similarity index 100%
rename from core/build.rs
rename to native/core/build.rs
diff --git a/core/rustfmt.toml b/native/core/rustfmt.toml
similarity index 100%
rename from core/rustfmt.toml
rename to native/core/rustfmt.toml
diff --git a/core/src/common/bit.rs b/native/core/src/common/bit.rs
similarity index 100%
rename from core/src/common/bit.rs
rename to native/core/src/common/bit.rs
diff --git a/core/src/common/buffer.rs b/native/core/src/common/buffer.rs
similarity index 100%
rename from core/src/common/buffer.rs
rename to native/core/src/common/buffer.rs
diff --git a/core/src/common/mod.rs b/native/core/src/common/mod.rs
similarity index 100%
rename from core/src/common/mod.rs
rename to native/core/src/common/mod.rs
diff --git a/core/src/data_type.rs b/native/core/src/data_type.rs
similarity index 100%
rename from core/src/data_type.rs
rename to native/core/src/data_type.rs
diff --git a/core/src/errors.rs b/native/core/src/errors.rs
similarity index 99%
rename from core/src/errors.rs
rename to native/core/src/errors.rs
index 09875bd9f..8c02a72d1 100644
--- a/core/src/errors.rs
+++ b/native/core/src/errors.rs
@@ -551,7 +551,7 @@ mod tests {
INIT.call_once(|| {
// Add common classes to the classpath in so that we can find CometException
let mut common_classes = PathBuf::from(env!("CARGO_MANIFEST_DIR"));
- common_classes.push("../common/target/classes");
+ common_classes.push("../../common/target/classes");
let mut class_path = common_classes
.as_path()
.to_str()
diff --git a/core/src/execution/datafusion/expressions/avg.rs b/native/core/src/execution/datafusion/expressions/avg.rs
similarity index 99%
rename from core/src/execution/datafusion/expressions/avg.rs
rename to native/core/src/execution/datafusion/expressions/avg.rs
index 1ff276e5d..3c8865bd1 100644
--- a/core/src/execution/datafusion/expressions/avg.rs
+++ b/native/core/src/execution/datafusion/expressions/avg.rs
@@ -47,7 +47,7 @@ pub struct Avg {
impl Avg {
/// Create a new AVG aggregate function
pub fn new(expr: Arc, name: impl Into, data_type: DataType) -> Self {
- let result_data_type = avg_return_type(&data_type).unwrap();
+ let result_data_type = avg_return_type("avg", &data_type).unwrap();
Self {
name: name.into(),
diff --git a/core/src/execution/datafusion/expressions/avg_decimal.rs b/native/core/src/execution/datafusion/expressions/avg_decimal.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/avg_decimal.rs
rename to native/core/src/execution/datafusion/expressions/avg_decimal.rs
diff --git a/core/src/execution/datafusion/expressions/bitwise_not.rs b/native/core/src/execution/datafusion/expressions/bitwise_not.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/bitwise_not.rs
rename to native/core/src/execution/datafusion/expressions/bitwise_not.rs
diff --git a/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs
rename to native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs
diff --git a/core/src/execution/datafusion/expressions/cast.rs b/native/core/src/execution/datafusion/expressions/cast.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/cast.rs
rename to native/core/src/execution/datafusion/expressions/cast.rs
diff --git a/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/checkoverflow.rs
rename to native/core/src/execution/datafusion/expressions/checkoverflow.rs
diff --git a/core/src/execution/datafusion/expressions/correlation.rs b/native/core/src/execution/datafusion/expressions/correlation.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/correlation.rs
rename to native/core/src/execution/datafusion/expressions/correlation.rs
diff --git a/core/src/execution/datafusion/expressions/covariance.rs b/native/core/src/execution/datafusion/expressions/covariance.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/covariance.rs
rename to native/core/src/execution/datafusion/expressions/covariance.rs
diff --git a/core/src/execution/datafusion/expressions/create_named_struct.rs b/native/core/src/execution/datafusion/expressions/create_named_struct.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/create_named_struct.rs
rename to native/core/src/execution/datafusion/expressions/create_named_struct.rs
diff --git a/core/src/execution/datafusion/expressions/if_expr.rs b/native/core/src/execution/datafusion/expressions/if_expr.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/if_expr.rs
rename to native/core/src/execution/datafusion/expressions/if_expr.rs
diff --git a/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs
similarity index 70%
rename from core/src/execution/datafusion/expressions/mod.rs
rename to native/core/src/execution/datafusion/expressions/mod.rs
index 385857f1e..98b422dce 100644
--- a/core/src/execution/datafusion/expressions/mod.rs
+++ b/native/core/src/execution/datafusion/expressions/mod.rs
@@ -24,10 +24,8 @@ pub mod if_expr;
mod normalize_nan;
pub mod scalar_funcs;
pub use normalize_nan::NormalizeNaNAndZero;
-use prost::DecodeError;
-use crate::{errors::CometError, execution::spark_expression};
-pub mod abs;
+use crate::errors::CometError;
pub mod avg;
pub mod avg_decimal;
pub mod bloom_filter_might_contain;
@@ -46,24 +44,7 @@ mod utils;
pub mod variance;
pub mod xxhash64;
-#[derive(Debug, Hash, PartialEq, Clone, Copy)]
-pub enum EvalMode {
- Legacy,
- Ansi,
- Try,
-}
-
-impl TryFrom for EvalMode {
- type Error = DecodeError;
-
- fn try_from(value: i32) -> Result {
- match spark_expression::EvalMode::try_from(value)? {
- spark_expression::EvalMode::Legacy => Ok(EvalMode::Legacy),
- spark_expression::EvalMode::Try => Ok(EvalMode::Try),
- spark_expression::EvalMode::Ansi => Ok(EvalMode::Ansi),
- }
- }
-}
+pub use datafusion_comet_spark_expr::EvalMode;
fn arithmetic_overflow_error(from_type: &str) -> CometError {
CometError::ArithmeticOverflow {
diff --git a/core/src/execution/datafusion/expressions/negative.rs b/native/core/src/execution/datafusion/expressions/negative.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/negative.rs
rename to native/core/src/execution/datafusion/expressions/negative.rs
diff --git a/core/src/execution/datafusion/expressions/normalize_nan.rs b/native/core/src/execution/datafusion/expressions/normalize_nan.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/normalize_nan.rs
rename to native/core/src/execution/datafusion/expressions/normalize_nan.rs
diff --git a/core/src/execution/datafusion/expressions/scalar_funcs.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/scalar_funcs.rs
rename to native/core/src/execution/datafusion/expressions/scalar_funcs.rs
diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/chr.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/chr.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/scalar_funcs/chr.rs
rename to native/core/src/execution/datafusion/expressions/scalar_funcs/chr.rs
diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs
rename to native/core/src/execution/datafusion/expressions/scalar_funcs/hash_expressions.rs
diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/scalar_funcs/hex.rs
rename to native/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs
diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs b/native/core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs
rename to native/core/src/execution/datafusion/expressions/scalar_funcs/unhex.rs
diff --git a/core/src/execution/datafusion/expressions/stats.rs b/native/core/src/execution/datafusion/expressions/stats.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/stats.rs
rename to native/core/src/execution/datafusion/expressions/stats.rs
diff --git a/core/src/execution/datafusion/expressions/stddev.rs b/native/core/src/execution/datafusion/expressions/stddev.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/stddev.rs
rename to native/core/src/execution/datafusion/expressions/stddev.rs
diff --git a/core/src/execution/datafusion/expressions/strings.rs b/native/core/src/execution/datafusion/expressions/strings.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/strings.rs
rename to native/core/src/execution/datafusion/expressions/strings.rs
diff --git a/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/datafusion/expressions/subquery.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/subquery.rs
rename to native/core/src/execution/datafusion/expressions/subquery.rs
diff --git a/core/src/execution/datafusion/expressions/sum_decimal.rs b/native/core/src/execution/datafusion/expressions/sum_decimal.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/sum_decimal.rs
rename to native/core/src/execution/datafusion/expressions/sum_decimal.rs
diff --git a/core/src/execution/datafusion/expressions/temporal.rs b/native/core/src/execution/datafusion/expressions/temporal.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/temporal.rs
rename to native/core/src/execution/datafusion/expressions/temporal.rs
diff --git a/core/src/execution/datafusion/expressions/unbound.rs b/native/core/src/execution/datafusion/expressions/unbound.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/unbound.rs
rename to native/core/src/execution/datafusion/expressions/unbound.rs
diff --git a/core/src/execution/datafusion/expressions/utils.rs b/native/core/src/execution/datafusion/expressions/utils.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/utils.rs
rename to native/core/src/execution/datafusion/expressions/utils.rs
diff --git a/core/src/execution/datafusion/expressions/variance.rs b/native/core/src/execution/datafusion/expressions/variance.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/variance.rs
rename to native/core/src/execution/datafusion/expressions/variance.rs
diff --git a/core/src/execution/datafusion/expressions/xxhash64.rs b/native/core/src/execution/datafusion/expressions/xxhash64.rs
similarity index 100%
rename from core/src/execution/datafusion/expressions/xxhash64.rs
rename to native/core/src/execution/datafusion/expressions/xxhash64.rs
diff --git a/core/src/execution/datafusion/mod.rs b/native/core/src/execution/datafusion/mod.rs
similarity index 100%
rename from core/src/execution/datafusion/mod.rs
rename to native/core/src/execution/datafusion/mod.rs
diff --git a/core/src/execution/datafusion/operators/expand.rs b/native/core/src/execution/datafusion/operators/expand.rs
similarity index 99%
rename from core/src/execution/datafusion/operators/expand.rs
rename to native/core/src/execution/datafusion/operators/expand.rs
index 5285dfb46..67171212f 100644
--- a/core/src/execution/datafusion/operators/expand.rs
+++ b/native/core/src/execution/datafusion/operators/expand.rs
@@ -126,6 +126,10 @@ impl ExecutionPlan for CometExpandExec {
fn properties(&self) -> &PlanProperties {
&self.cache
}
+
+ fn name(&self) -> &str {
+ "CometExpandExec"
+ }
}
pub struct ExpandStream {
diff --git a/core/src/execution/datafusion/operators/mod.rs b/native/core/src/execution/datafusion/operators/mod.rs
similarity index 100%
rename from core/src/execution/datafusion/operators/mod.rs
rename to native/core/src/execution/datafusion/operators/mod.rs
diff --git a/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs
similarity index 95%
rename from core/src/execution/datafusion/planner.rs
rename to native/core/src/execution/datafusion/planner.rs
index 8d7d24654..ee208ac74 100644
--- a/core/src/execution/datafusion/planner.rs
+++ b/native/core/src/execution/datafusion/planner.rs
@@ -20,19 +20,22 @@
use std::{collections::HashMap, sync::Arc};
use arrow_schema::{DataType, Field, Schema, TimeUnit};
+use datafusion::functions_aggregate::bit_and_or_xor::{bit_and_udaf, bit_or_udaf, bit_xor_udaf};
+use datafusion::functions_aggregate::count::count_udaf;
+use datafusion::functions_aggregate::sum::sum_udaf;
use datafusion::physical_plan::windows::BoundedWindowAggExec;
use datafusion::physical_plan::InputOrderMode;
use datafusion::{
arrow::{compute::SortOptions, datatypes::SchemaRef},
common::DataFusionError,
execution::FunctionRegistry,
+ functions_aggregate::first_last::{FirstValue, LastValue},
logical_expr::Operator as DataFusionOperator,
physical_expr::{
execution_props::ExecutionProps,
expressions::{
- in_list, BinaryExpr, BitAnd, BitOr, BitXor, CaseExpr, CastExpr, Column, Count,
- FirstValue, IsNotNullExpr, IsNullExpr, LastValue, Literal as DataFusionLiteral, Max,
- Min, NotExpr, Sum,
+ in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr,
+ Literal as DataFusionLiteral, Max, Min, NotExpr,
},
AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr,
},
@@ -104,7 +107,8 @@ use crate::{
},
};
-use super::expressions::{abs::CometAbsFunc, create_named_struct::CreateNamedStruct, EvalMode};
+use super::expressions::{create_named_struct::CreateNamedStruct, EvalMode};
+use datafusion_comet_spark_expr::abs::Abs;
// For clippy error on type_complexity.
type ExecResult = Result;
@@ -365,7 +369,7 @@ impl PhysicalPlanner {
let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?;
let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap());
let timezone = expr.timezone.clone();
- let eval_mode = expr.eval_mode.try_into()?;
+ let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?;
Ok(Arc::new(Cast::new(child, datatype, eval_mode, timezone)))
}
@@ -504,8 +508,8 @@ impl PhysicalPlanner {
let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?;
let return_type = child.data_type(&input_schema)?;
let args = vec![child];
- let eval_mode = expr.eval_mode.try_into()?;
- let comet_abs = Arc::new(ScalarUDF::new_from_impl(CometAbsFunc::new(
+ let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?;
+ let comet_abs = Arc::new(ScalarUDF::new_from_impl(Abs::new(
eval_mode,
return_type.to_string(),
)?));
@@ -647,7 +651,7 @@ impl PhysicalPlanner {
let left = self.create_expr(left, input_schema.clone())?;
let right = self.create_expr(right, input_schema.clone())?;
match (
- op,
+ &op,
left.data_type(&input_schema),
right.data_type(&input_schema),
) {
@@ -916,7 +920,7 @@ impl PhysicalPlanner {
// the data corruption. Note that we only need to copy the input batch
// if the child operator is `ScanExec`, because other operators after `ScanExec`
// will create new arrays for the output batch.
- let child = if child.as_any().downcast_ref::().is_some() {
+ let child = if child.as_any().is::() {
Arc::new(CopyExec::new(child))
} else {
child
@@ -1208,11 +1212,19 @@ impl PhysicalPlanner {
.iter()
.map(|child| self.create_expr(child, schema.clone()))
.collect::, _>>()?;
- Ok(Arc::new(Count::new_with_multiple_exprs(
- children,
+
+ create_aggregate_expr(
+ &count_udaf(),
+ &children,
+ &[],
+ &[],
+ &[],
+ schema.as_ref(),
"count",
- DataType::Int64,
- )))
+ false,
+ false,
+ )
+ .map_err(|e| ExecutionError::DataFusionError(e.to_string()))
}
AggExprStruct::Min(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?;
@@ -1236,7 +1248,18 @@ impl PhysicalPlanner {
// cast to the result data type of SUM if necessary, we should not expect
// a cast failure since it should have already been checked at Spark side
let child = Arc::new(CastExpr::new(child, datatype.clone(), None));
- Ok(Arc::new(Sum::new(child, "sum", datatype)))
+ create_aggregate_expr(
+ &sum_udaf(),
+ &[child],
+ &[],
+ &[],
+ &[],
+ schema.as_ref(),
+ "sum",
+ false,
+ false,
+ )
+ .map_err(|e| ExecutionError::DataFusionError(e.to_string()))
}
}
}
@@ -1263,31 +1286,79 @@ impl PhysicalPlanner {
AggExprStruct::First(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?;
let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new());
-
- create_aggregate_expr(&func, &[child], &[], &[], &schema, "first", false, false)
- .map_err(|e| e.into())
+ create_aggregate_expr(
+ &func,
+ &[child],
+ &[],
+ &[],
+ &[],
+ &schema,
+ "first",
+ false,
+ false,
+ )
+ .map_err(|e| e.into())
}
AggExprStruct::Last(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?;
let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new());
-
- create_aggregate_expr(&func, &[child], &[], &[], &schema, "last", false, false)
- .map_err(|e| e.into())
+ create_aggregate_expr(
+ &func,
+ &[child],
+ &[],
+ &[],
+ &[],
+ &schema,
+ "last",
+ false,
+ false,
+ )
+ .map_err(|e| e.into())
}
AggExprStruct::BitAndAgg(expr) => {
- let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?;
- let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap());
- Ok(Arc::new(BitAnd::new(child, "bit_and", datatype)))
+ let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?;
+ create_aggregate_expr(
+ &bit_and_udaf(),
+ &[child],
+ &[],
+ &[],
+ &[],
+ &schema,
+ "bit_and",
+ false,
+ false,
+ )
+ .map_err(|e| e.into())
}
AggExprStruct::BitOrAgg(expr) => {
- let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?;
- let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap());
- Ok(Arc::new(BitOr::new(child, "bit_or", datatype)))
+ let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?;
+ create_aggregate_expr(
+ &bit_or_udaf(),
+ &[child],
+ &[],
+ &[],
+ &[],
+ &schema,
+ "bit_or",
+ false,
+ false,
+ )
+ .map_err(|e| e.into())
}
AggExprStruct::BitXorAgg(expr) => {
- let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?;
- let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap());
- Ok(Arc::new(BitXor::new(child, "bit_xor", datatype)))
+ let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?;
+ create_aggregate_expr(
+ &bit_xor_udaf(),
+ &[child],
+ &[],
+ &[],
+ &[],
+ &schema,
+ "bit_xor",
+ false,
+ false,
+ )
+ .map_err(|e| e.into())
}
AggExprStruct::Covariance(expr) => {
let child1 = self.create_expr(expr.child1.as_ref().unwrap(), schema.clone())?;
@@ -1483,6 +1554,7 @@ impl PhysicalPlanner {
&window_func,
window_func_name,
&window_args,
+ &[],
partition_by,
sort_exprs,
window_frame.into(),
@@ -1609,10 +1681,10 @@ impl From for DataFusionError {
/// modification. This is used to determine if we need to copy the input batch to avoid
/// data corruption from reusing the input batch.
fn can_reuse_input_batch(op: &Arc) -> bool {
- op.as_any().downcast_ref::().is_some()
- || op.as_any().downcast_ref::().is_some()
- || op.as_any().downcast_ref::().is_some()
- || op.as_any().downcast_ref::().is_some()
+ op.as_any().is::()
+ || op.as_any().is::()
+ || op.as_any().is::()
+ || op.as_any().is::()
}
/// Collects the indices of the columns in the input schema that are used in the expression
@@ -1743,6 +1815,14 @@ fn rewrite_physical_expr(
Ok(expr.rewrite(&mut rewriter).data()?)
}
+fn from_protobuf_eval_mode(value: i32) -> Result {
+ match spark_expression::EvalMode::try_from(value)? {
+ spark_expression::EvalMode::Legacy => Ok(EvalMode::Legacy),
+ spark_expression::EvalMode::Try => Ok(EvalMode::Try),
+ spark_expression::EvalMode::Ansi => Ok(EvalMode::Ansi),
+ }
+}
+
#[cfg(test)]
mod tests {
use std::{sync::Arc, task::Poll};
diff --git a/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/datafusion/shuffle_writer.rs
similarity index 99%
rename from core/src/execution/datafusion/shuffle_writer.rs
rename to native/core/src/execution/datafusion/shuffle_writer.rs
index 637cb6d51..3b934813d 100644
--- a/core/src/execution/datafusion/shuffle_writer.rs
+++ b/native/core/src/execution/datafusion/shuffle_writer.rs
@@ -160,6 +160,10 @@ impl ExecutionPlan for ShuffleWriterExec {
fn properties(&self) -> &PlanProperties {
&self.cache
}
+
+ fn name(&self) -> &str {
+ "ShuffleWriterExec"
+ }
}
impl ShuffleWriterExec {
diff --git a/core/src/execution/datafusion/spark_hash.rs b/native/core/src/execution/datafusion/spark_hash.rs
similarity index 100%
rename from core/src/execution/datafusion/spark_hash.rs
rename to native/core/src/execution/datafusion/spark_hash.rs
diff --git a/core/src/execution/datafusion/util/mod.rs b/native/core/src/execution/datafusion/util/mod.rs
similarity index 100%
rename from core/src/execution/datafusion/util/mod.rs
rename to native/core/src/execution/datafusion/util/mod.rs
diff --git a/core/src/execution/datafusion/util/spark_bit_array.rs b/native/core/src/execution/datafusion/util/spark_bit_array.rs
similarity index 100%
rename from core/src/execution/datafusion/util/spark_bit_array.rs
rename to native/core/src/execution/datafusion/util/spark_bit_array.rs
diff --git a/core/src/execution/datafusion/util/spark_bloom_filter.rs b/native/core/src/execution/datafusion/util/spark_bloom_filter.rs
similarity index 100%
rename from core/src/execution/datafusion/util/spark_bloom_filter.rs
rename to native/core/src/execution/datafusion/util/spark_bloom_filter.rs
diff --git a/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs
similarity index 100%
rename from core/src/execution/jni_api.rs
rename to native/core/src/execution/jni_api.rs
diff --git a/core/src/execution/kernels/hash.rs b/native/core/src/execution/kernels/hash.rs
similarity index 100%
rename from core/src/execution/kernels/hash.rs
rename to native/core/src/execution/kernels/hash.rs
diff --git a/core/src/execution/kernels/mod.rs b/native/core/src/execution/kernels/mod.rs
similarity index 100%
rename from core/src/execution/kernels/mod.rs
rename to native/core/src/execution/kernels/mod.rs
diff --git a/core/src/execution/kernels/strings.rs b/native/core/src/execution/kernels/strings.rs
similarity index 100%
rename from core/src/execution/kernels/strings.rs
rename to native/core/src/execution/kernels/strings.rs
diff --git a/core/src/execution/kernels/temporal.rs b/native/core/src/execution/kernels/temporal.rs
similarity index 100%
rename from core/src/execution/kernels/temporal.rs
rename to native/core/src/execution/kernels/temporal.rs
diff --git a/core/src/execution/memory_pool.rs b/native/core/src/execution/memory_pool.rs
similarity index 100%
rename from core/src/execution/memory_pool.rs
rename to native/core/src/execution/memory_pool.rs
diff --git a/core/src/execution/metrics/mod.rs b/native/core/src/execution/metrics/mod.rs
similarity index 100%
rename from core/src/execution/metrics/mod.rs
rename to native/core/src/execution/metrics/mod.rs
diff --git a/core/src/execution/metrics/utils.rs b/native/core/src/execution/metrics/utils.rs
similarity index 100%
rename from core/src/execution/metrics/utils.rs
rename to native/core/src/execution/metrics/utils.rs
diff --git a/core/src/execution/mod.rs b/native/core/src/execution/mod.rs
similarity index 100%
rename from core/src/execution/mod.rs
rename to native/core/src/execution/mod.rs
diff --git a/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs
similarity index 98%
rename from core/src/execution/operators/copy.rs
rename to native/core/src/execution/operators/copy.rs
index d011b3cb2..68c91aafc 100644
--- a/core/src/execution/operators/copy.rs
+++ b/native/core/src/execution/operators/copy.rs
@@ -126,6 +126,10 @@ impl ExecutionPlan for CopyExec {
fn properties(&self) -> &PlanProperties {
&self.cache
}
+
+ fn name(&self) -> &str {
+ "CopyExec"
+ }
}
struct CopyStream {
diff --git a/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs
similarity index 100%
rename from core/src/execution/operators/mod.rs
rename to native/core/src/execution/operators/mod.rs
diff --git a/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs
similarity index 99%
rename from core/src/execution/operators/scan.rs
rename to native/core/src/execution/operators/scan.rs
index de5328210..68dd773cf 100644
--- a/core/src/execution/operators/scan.rs
+++ b/native/core/src/execution/operators/scan.rs
@@ -270,6 +270,10 @@ impl ExecutionPlan for ScanExec {
fn properties(&self) -> &PlanProperties {
&self.cache
}
+
+ fn name(&self) -> &str {
+ "ScanExec"
+ }
}
impl DisplayAs for ScanExec {
diff --git a/core/src/execution/proto/expr.proto b/native/core/src/execution/proto/expr.proto
similarity index 100%
rename from core/src/execution/proto/expr.proto
rename to native/core/src/execution/proto/expr.proto
diff --git a/core/src/execution/proto/operator.proto b/native/core/src/execution/proto/operator.proto
similarity index 100%
rename from core/src/execution/proto/operator.proto
rename to native/core/src/execution/proto/operator.proto
diff --git a/core/src/execution/proto/partitioning.proto b/native/core/src/execution/proto/partitioning.proto
similarity index 100%
rename from core/src/execution/proto/partitioning.proto
rename to native/core/src/execution/proto/partitioning.proto
diff --git a/core/src/execution/serde.rs b/native/core/src/execution/serde.rs
similarity index 100%
rename from core/src/execution/serde.rs
rename to native/core/src/execution/serde.rs
diff --git a/core/src/execution/shuffle/list.rs b/native/core/src/execution/shuffle/list.rs
similarity index 100%
rename from core/src/execution/shuffle/list.rs
rename to native/core/src/execution/shuffle/list.rs
diff --git a/core/src/execution/shuffle/map.rs b/native/core/src/execution/shuffle/map.rs
similarity index 100%
rename from core/src/execution/shuffle/map.rs
rename to native/core/src/execution/shuffle/map.rs
diff --git a/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs
similarity index 100%
rename from core/src/execution/shuffle/mod.rs
rename to native/core/src/execution/shuffle/mod.rs
diff --git a/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs
similarity index 100%
rename from core/src/execution/shuffle/row.rs
rename to native/core/src/execution/shuffle/row.rs
diff --git a/core/src/execution/sort.rs b/native/core/src/execution/sort.rs
similarity index 100%
rename from core/src/execution/sort.rs
rename to native/core/src/execution/sort.rs
diff --git a/core/src/execution/timezone.rs b/native/core/src/execution/timezone.rs
similarity index 100%
rename from core/src/execution/timezone.rs
rename to native/core/src/execution/timezone.rs
diff --git a/core/src/execution/utils.rs b/native/core/src/execution/utils.rs
similarity index 100%
rename from core/src/execution/utils.rs
rename to native/core/src/execution/utils.rs
diff --git a/core/src/jvm_bridge/batch_iterator.rs b/native/core/src/jvm_bridge/batch_iterator.rs
similarity index 100%
rename from core/src/jvm_bridge/batch_iterator.rs
rename to native/core/src/jvm_bridge/batch_iterator.rs
diff --git a/core/src/jvm_bridge/comet_exec.rs b/native/core/src/jvm_bridge/comet_exec.rs
similarity index 100%
rename from core/src/jvm_bridge/comet_exec.rs
rename to native/core/src/jvm_bridge/comet_exec.rs
diff --git a/core/src/jvm_bridge/comet_metric_node.rs b/native/core/src/jvm_bridge/comet_metric_node.rs
similarity index 100%
rename from core/src/jvm_bridge/comet_metric_node.rs
rename to native/core/src/jvm_bridge/comet_metric_node.rs
diff --git a/core/src/jvm_bridge/comet_task_memory_manager.rs b/native/core/src/jvm_bridge/comet_task_memory_manager.rs
similarity index 100%
rename from core/src/jvm_bridge/comet_task_memory_manager.rs
rename to native/core/src/jvm_bridge/comet_task_memory_manager.rs
diff --git a/core/src/jvm_bridge/mod.rs b/native/core/src/jvm_bridge/mod.rs
similarity index 100%
rename from core/src/jvm_bridge/mod.rs
rename to native/core/src/jvm_bridge/mod.rs
diff --git a/core/src/lib.rs b/native/core/src/lib.rs
similarity index 100%
rename from core/src/lib.rs
rename to native/core/src/lib.rs
diff --git a/core/src/parquet/compression.rs b/native/core/src/parquet/compression.rs
similarity index 100%
rename from core/src/parquet/compression.rs
rename to native/core/src/parquet/compression.rs
diff --git a/core/src/parquet/data_type.rs b/native/core/src/parquet/data_type.rs
similarity index 100%
rename from core/src/parquet/data_type.rs
rename to native/core/src/parquet/data_type.rs
diff --git a/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs
similarity index 100%
rename from core/src/parquet/mod.rs
rename to native/core/src/parquet/mod.rs
diff --git a/core/src/parquet/mutable_vector.rs b/native/core/src/parquet/mutable_vector.rs
similarity index 100%
rename from core/src/parquet/mutable_vector.rs
rename to native/core/src/parquet/mutable_vector.rs
diff --git a/core/src/parquet/read/column.rs b/native/core/src/parquet/read/column.rs
similarity index 100%
rename from core/src/parquet/read/column.rs
rename to native/core/src/parquet/read/column.rs
diff --git a/core/src/parquet/read/levels.rs b/native/core/src/parquet/read/levels.rs
similarity index 100%
rename from core/src/parquet/read/levels.rs
rename to native/core/src/parquet/read/levels.rs
diff --git a/core/src/parquet/read/mod.rs b/native/core/src/parquet/read/mod.rs
similarity index 100%
rename from core/src/parquet/read/mod.rs
rename to native/core/src/parquet/read/mod.rs
diff --git a/core/src/parquet/read/values.rs b/native/core/src/parquet/read/values.rs
similarity index 100%
rename from core/src/parquet/read/values.rs
rename to native/core/src/parquet/read/values.rs
diff --git a/core/src/parquet/util/bit_packing.rs b/native/core/src/parquet/util/bit_packing.rs
similarity index 100%
rename from core/src/parquet/util/bit_packing.rs
rename to native/core/src/parquet/util/bit_packing.rs
diff --git a/core/src/parquet/util/buffer.rs b/native/core/src/parquet/util/buffer.rs
similarity index 100%
rename from core/src/parquet/util/buffer.rs
rename to native/core/src/parquet/util/buffer.rs
diff --git a/core/src/parquet/util/jni.rs b/native/core/src/parquet/util/jni.rs
similarity index 100%
rename from core/src/parquet/util/jni.rs
rename to native/core/src/parquet/util/jni.rs
diff --git a/core/src/parquet/util/jni_buffer.rs b/native/core/src/parquet/util/jni_buffer.rs
similarity index 100%
rename from core/src/parquet/util/jni_buffer.rs
rename to native/core/src/parquet/util/jni_buffer.rs
diff --git a/core/src/parquet/util/memory.rs b/native/core/src/parquet/util/memory.rs
similarity index 100%
rename from core/src/parquet/util/memory.rs
rename to native/core/src/parquet/util/memory.rs
diff --git a/core/src/parquet/util/mod.rs b/native/core/src/parquet/util/mod.rs
similarity index 100%
rename from core/src/parquet/util/mod.rs
rename to native/core/src/parquet/util/mod.rs
diff --git a/core/src/parquet/util/test_common/file_util.rs b/native/core/src/parquet/util/test_common/file_util.rs
similarity index 100%
rename from core/src/parquet/util/test_common/file_util.rs
rename to native/core/src/parquet/util/test_common/file_util.rs
diff --git a/core/src/parquet/util/test_common/mod.rs b/native/core/src/parquet/util/test_common/mod.rs
similarity index 100%
rename from core/src/parquet/util/test_common/mod.rs
rename to native/core/src/parquet/util/test_common/mod.rs
diff --git a/core/src/parquet/util/test_common/page_util.rs b/native/core/src/parquet/util/test_common/page_util.rs
similarity index 100%
rename from core/src/parquet/util/test_common/page_util.rs
rename to native/core/src/parquet/util/test_common/page_util.rs
diff --git a/core/src/parquet/util/test_common/rand_gen.rs b/native/core/src/parquet/util/test_common/rand_gen.rs
similarity index 100%
rename from core/src/parquet/util/test_common/rand_gen.rs
rename to native/core/src/parquet/util/test_common/rand_gen.rs
diff --git a/core/testdata/backtrace.txt b/native/core/testdata/backtrace.txt
similarity index 100%
rename from core/testdata/backtrace.txt
rename to native/core/testdata/backtrace.txt
diff --git a/core/testdata/stacktrace.txt b/native/core/testdata/stacktrace.txt
similarity index 100%
rename from core/testdata/stacktrace.txt
rename to native/core/testdata/stacktrace.txt
diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml
new file mode 100644
index 000000000..d10d04944
--- /dev/null
+++ b/native/spark-expr/Cargo.toml
@@ -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]
+name = "datafusion-comet-spark-expr"
+description = "DataFusion expressions that emulate Apache Spark's behavior"
+version = { workspace = true }
+homepage = { workspace = true }
+repository = { workspace = true }
+authors = { workspace = true }
+readme = { workspace = true }
+license = { workspace = true }
+edition = { workspace = true }
+
+[dependencies]
+arrow = { workspace = true }
+arrow-schema = { workspace = true }
+datafusion = { workspace = true }
+datafusion-common = { workspace = true }
+datafusion-functions = { workspace = true }
+
+[lib]
+name = "datafusion_comet_spark_expr"
+path = "src/lib.rs"
diff --git a/native/spark-expr/README.md b/native/spark-expr/README.md
new file mode 100644
index 000000000..a7ee75363
--- /dev/null
+++ b/native/spark-expr/README.md
@@ -0,0 +1,23 @@
+
+
+# datafusion-comet-spark-expr: Spark-compatible Expressions
+
+This crate provides Apache Spark-compatible expressions for use with DataFusion and is maintained as part of the
+[Apache DataFusion Comet](https://github.com/apache/datafusion-comet/) subproject.
\ No newline at end of file
diff --git a/core/src/execution/datafusion/expressions/abs.rs b/native/spark-expr/src/abs.rs
similarity index 79%
rename from core/src/execution/datafusion/expressions/abs.rs
rename to native/spark-expr/src/abs.rs
index 4eb8c7c1e..198a96e57 100644
--- a/core/src/execution/datafusion/expressions/abs.rs
+++ b/native/spark-expr/src/abs.rs
@@ -15,34 +15,37 @@
// specific language governing permissions and limitations
// under the License.
+//! Spark-compatible implementation of abs function
+
+use std::{any::Any, sync::Arc};
+
use arrow::datatypes::DataType;
use arrow_schema::ArrowError;
+
use datafusion::logical_expr::{ColumnarValue, ScalarUDFImpl, Signature};
use datafusion_common::DataFusionError;
use datafusion_functions::math;
-use std::{any::Any, sync::Arc};
-
-use crate::execution::operators::ExecutionError;
-use super::{arithmetic_overflow_error, EvalMode};
+use super::{EvalMode, SparkError};
+/// Spark-compatible ABS expression
#[derive(Debug)]
-pub struct CometAbsFunc {
+pub struct Abs {
inner_abs_func: Arc,
eval_mode: EvalMode,
data_type_name: String,
}
-impl CometAbsFunc {
- pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result {
+impl Abs {
+ pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result {
if let EvalMode::Legacy | EvalMode::Ansi = eval_mode {
Ok(Self {
- inner_abs_func: math::abs().inner(),
+ inner_abs_func: math::abs().inner().clone(),
eval_mode,
data_type_name,
})
} else {
- Err(ExecutionError::GeneralError(format!(
+ Err(DataFusionError::Execution(format!(
"Invalid EvalMode: \"{:?}\"",
eval_mode
)))
@@ -50,7 +53,7 @@ impl CometAbsFunc {
}
}
-impl ScalarUDFImpl for CometAbsFunc {
+impl ScalarUDFImpl for Abs {
fn as_any(&self) -> &dyn Any {
self
}
@@ -68,17 +71,15 @@ impl ScalarUDFImpl for CometAbsFunc {
fn invoke(&self, args: &[ColumnarValue]) -> Result {
match self.inner_abs_func.invoke(args) {
- Err(DataFusionError::ArrowError(ArrowError::ComputeError(msg), trace))
+ Err(DataFusionError::ArrowError(ArrowError::ComputeError(msg), _))
if msg.contains("overflow") =>
{
if self.eval_mode == EvalMode::Legacy {
Ok(args[0].clone())
} else {
- let msg = arithmetic_overflow_error(&self.data_type_name).to_string();
- Err(DataFusionError::ArrowError(
- ArrowError::ComputeError(msg),
- trace,
- ))
+ Err(DataFusionError::External(Box::new(
+ SparkError::ArithmeticOverflow(self.data_type_name.clone()),
+ )))
}
}
other => other,
diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs
new file mode 100644
index 000000000..3873754be
--- /dev/null
+++ b/native/spark-expr/src/lib.rs
@@ -0,0 +1,56 @@
+// 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.
+
+use std::error::Error;
+use std::fmt::{Display, Formatter};
+
+pub mod abs;
+
+/// Spark supports three evaluation modes when evaluating expressions, which affect
+/// the behavior when processing input values that are invalid or would result in an
+/// error, such as divide by zero errors, and also affects behavior when converting
+/// between types.
+#[derive(Debug, Hash, PartialEq, Clone, Copy)]
+pub enum EvalMode {
+ /// Legacy is the default behavior in Spark prior to Spark 4.0. This mode silently ignores
+ /// or replaces errors during SQL operations. Operations resulting in errors (like
+ /// division by zero) will produce NULL values instead of failing. Legacy mode also
+ /// enables implicit type conversions.
+ Legacy,
+ /// Adheres to the ANSI SQL standard for error handling by throwing exceptions for
+ /// operations that result in errors. Does not perform implicit type conversions.
+ Ansi,
+ /// Same as Ansi mode, except that it converts errors to NULL values without
+ /// failing the entire query.
+ Try,
+}
+
+#[derive(Debug)]
+pub enum SparkError {
+ ArithmeticOverflow(String),
+}
+
+impl Error for SparkError {}
+
+impl Display for SparkError {
+ fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+ match self {
+ Self::ArithmeticOverflow(data_type) =>
+ write!(f, "[ARITHMETIC_OVERFLOW] {} overflow. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.", data_type)
+ }
+ }
+}
diff --git a/pom.xml b/pom.xml
index 043c11e89..fcea91cf7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -59,7 +59,7 @@ under the License.
0.8.11
4.8.8
2.0.6
- ${project.basedir}/../core/target/debug
+ ${project.basedir}/../native/target/debug
darwin
x86_64
org.apache.comet.shaded
@@ -454,7 +454,7 @@ under the License.
release
- ${project.basedir}/../core/target/release
+ ${project.basedir}/../native/target/release
diff --git a/spark/pom.xml b/spark/pom.xml
index 6b3b36aa5..49672e0bb 100644
--- a/spark/pom.xml
+++ b/spark/pom.xml
@@ -152,7 +152,7 @@ under the License.
com.google.protobuf:protoc:${protobuf.version}
- ../core/src/execution/proto
+ ../native/core/src/execution/proto
diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
index fb1f62542..c30d6b0f7 100644
--- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
+++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
@@ -331,8 +331,8 @@ class CometSparkSessionExtensions
CometProjectExec(
nativeOp,
op,
- op.projectList,
op.output,
+ op.projectList,
op.child,
SerializedPlan(None))
case None =>
@@ -343,7 +343,13 @@ class CometSparkSessionExtensions
val newOp = transform1(op)
newOp match {
case Some(nativeOp) =>
- CometFilterExec(nativeOp, op, op.condition, op.child, SerializedPlan(None))
+ CometFilterExec(
+ nativeOp,
+ op,
+ op.output,
+ op.condition,
+ op.child,
+ SerializedPlan(None))
case None =>
op
}
@@ -352,7 +358,14 @@ class CometSparkSessionExtensions
val newOp = transform1(op)
newOp match {
case Some(nativeOp) =>
- CometSortExec(nativeOp, op, op.sortOrder, op.child, SerializedPlan(None))
+ CometSortExec(
+ nativeOp,
+ op,
+ op.output,
+ op.outputOrdering,
+ op.sortOrder,
+ op.child,
+ SerializedPlan(None))
case None =>
op
}
@@ -393,12 +406,27 @@ class CometSparkSessionExtensions
val newOp = transform1(op)
newOp match {
case Some(nativeOp) =>
- CometExpandExec(nativeOp, op, op.projections, op.child, SerializedPlan(None))
+ CometExpandExec(
+ nativeOp,
+ op,
+ op.output,
+ op.projections,
+ op.child,
+ SerializedPlan(None))
case None =>
op
}
- case op @ HashAggregateExec(_, _, _, groupingExprs, aggExprs, _, _, _, child) =>
+ case op @ HashAggregateExec(
+ _,
+ _,
+ _,
+ groupingExprs,
+ aggExprs,
+ _,
+ _,
+ resultExpressions,
+ child) =>
val modes = aggExprs.map(_.mode).distinct
if (!modes.isEmpty && modes.size != 1) {
@@ -425,8 +453,10 @@ class CometSparkSessionExtensions
CometHashAggregateExec(
nativeOp,
op,
+ op.output,
groupingExprs,
aggExprs,
+ resultExpressions,
child.output,
if (modes.nonEmpty) Some(modes.head) else None,
child,
@@ -446,6 +476,8 @@ class CometSparkSessionExtensions
CometHashJoinExec(
nativeOp,
op,
+ op.output,
+ op.outputOrdering,
op.leftKeys,
op.rightKeys,
op.joinType,
@@ -478,6 +510,8 @@ class CometSparkSessionExtensions
CometBroadcastHashJoinExec(
nativeOp,
op,
+ op.output,
+ op.outputOrdering,
op.leftKeys,
op.rightKeys,
op.joinType,
@@ -499,6 +533,8 @@ class CometSparkSessionExtensions
CometSortMergeJoinExec(
nativeOp,
op,
+ op.output,
+ op.outputOrdering,
op.leftKeys,
op.rightKeys,
op.joinType,
@@ -535,7 +571,7 @@ class CometSparkSessionExtensions
&& isCometNative(child) =>
QueryPlanSerde.operator2Proto(c) match {
case Some(nativeOp) =>
- val cometOp = CometCoalesceExec(c, numPartitions, child)
+ val cometOp = CometCoalesceExec(c, c.output, numPartitions, child)
CometSinkPlaceHolder(nativeOp, c, cometOp)
case None =>
c
@@ -559,7 +595,13 @@ class CometSparkSessionExtensions
QueryPlanSerde.operator2Proto(s) match {
case Some(nativeOp) =>
val cometOp =
- CometTakeOrderedAndProjectExec(s, s.limit, s.sortOrder, s.projectList, s.child)
+ CometTakeOrderedAndProjectExec(
+ s,
+ s.output,
+ s.limit,
+ s.sortOrder,
+ s.projectList,
+ s.child)
CometSinkPlaceHolder(nativeOp, s, cometOp)
case None =>
s
@@ -580,7 +622,13 @@ class CometSparkSessionExtensions
newOp match {
case Some(nativeOp) =>
val cometOp =
- CometWindowExec(w, w.windowExpression, w.partitionSpec, w.orderSpec, w.child)
+ CometWindowExec(
+ w,
+ w.output,
+ w.windowExpression,
+ w.partitionSpec,
+ w.orderSpec,
+ w.child)
CometSinkPlaceHolder(nativeOp, w, cometOp)
case None =>
w
@@ -591,7 +639,7 @@ class CometSparkSessionExtensions
u.children.forall(isCometNative) =>
QueryPlanSerde.operator2Proto(u) match {
case Some(nativeOp) =>
- val cometOp = CometUnionExec(u, u.children)
+ val cometOp = CometUnionExec(u, u.output, u.children)
CometSinkPlaceHolder(nativeOp, u, cometOp)
case None =>
u
@@ -631,7 +679,7 @@ class CometSparkSessionExtensions
isSpark34Plus => // Spark 3.4+ only
QueryPlanSerde.operator2Proto(b) match {
case Some(nativeOp) =>
- val cometOp = CometBroadcastExchangeExec(b, b.child)
+ val cometOp = CometBroadcastExchangeExec(b, b.output, b.child)
CometSinkPlaceHolder(nativeOp, b, cometOp)
case None => b
}
diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
index 65de37c83..da534b02c 100644
--- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
+++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
@@ -208,7 +208,10 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim
expr match {
case agg: AggregateExpression =>
agg.aggregateFunction match {
- case _: Min | _: Max | _: Count =>
+ // TODO add support for Count (this was removed when upgrading
+ // to DataFusion 40 because it is no longer a built-in window function)
+ // https://github.com/apache/datafusion-comet/issues/645
+ case _: Min | _: Max =>
Some(agg)
case _ =>
withInfo(windowExpr, "Unsupported aggregate", expr)
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
index 38247b2c4..64d14a00e 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
@@ -31,6 +31,7 @@ import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec
import org.apache.spark.launcher.SparkLauncher
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution}
@@ -60,7 +61,10 @@ import org.apache.comet.CometRuntimeException
* Note that this only supports Spark 3.4 and later, because the serialization class
* `ChunkedByteBuffer` is only serializable in Spark 3.4 and later.
*/
-case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan)
+case class CometBroadcastExchangeExec(
+ originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
+ override val child: SparkPlan)
extends BroadcastExchangeLike
with ShimCometBroadcastExchangeExec {
import CometBroadcastExchangeExec._
@@ -75,7 +79,7 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan)
"broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast"))
override def doCanonicalize(): SparkPlan = {
- CometBroadcastExchangeExec(originalPlan.canonicalized, child.canonicalized)
+ CometBroadcastExchangeExec(null, null, child.canonicalized)
}
override def runtimeStatistics: Statistics = {
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala
index cc635d739..b216c7d81 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala
@@ -20,6 +20,7 @@
package org.apache.spark.sql.comet
import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition, UnknownPartitioning}
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -32,6 +33,7 @@ import com.google.common.base.Objects
*/
case class CometCoalesceExec(
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
numPartitions: Int,
child: SparkPlan)
extends CometExec
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala
index d2c9158ee..3effcca4f 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCollectLimitExec.scala
@@ -22,6 +22,8 @@ package org.apache.spark.sql.comet
import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition}
import org.apache.spark.sql.comet.execution.shuffle.{CometShuffledBatchRDD, CometShuffleExchangeExec}
import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec.{METRIC_NATIVE_TIME_DESCRIPTION, METRIC_NATIVE_TIME_NAME}
import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode, UnsafeRowSerializer}
@@ -45,6 +47,8 @@ case class CometCollectLimitExec(
child: SparkPlan)
extends CometExec
with UnaryExecNode {
+ override def output: Seq[Attribute] = child.output
+ override def outputPartitioning: Partitioning = SinglePartition
private lazy val writeMetrics =
SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext)
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala
index 6e9bfe424..3f09e2fe2 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometTakeOrderedAndProjectExec.scala
@@ -42,14 +42,13 @@ import org.apache.comet.shims.ShimCometTakeOrderedAndProjectExec
*/
case class CometTakeOrderedAndProjectExec(
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
limit: Int,
sortOrder: Seq[SortOrder],
projectList: Seq[NamedExpression],
child: SparkPlan)
extends CometExec
with UnaryExecNode {
- override def output: Seq[Attribute] = projectList.map(_.toAttribute)
-
private lazy val writeMetrics =
SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext)
private lazy val readMetrics =
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala
index 9685e75e1..4c10a8abb 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometWindowExec.scala
@@ -43,6 +43,7 @@ import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType, wi
*/
case class CometWindowExec(
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
windowExpression: Seq[NamedExpression],
partitionSpec: Seq[Expression],
orderSpec: Seq[SortOrder],
@@ -52,8 +53,6 @@ case class CometWindowExec(
override def nodeName: String = "CometWindowExec"
- override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute)
-
private lazy val writeMetrics =
SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext)
private lazy val readMetrics =
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala
index 8b1ba168f..1c9bf9947 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala
@@ -39,7 +39,6 @@ import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode
import org.apache.spark.sql.comet.util.Utils
import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec}
-import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.execution.exchange.ReusedExchangeExec
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.internal.SQLConf
@@ -154,7 +153,7 @@ abstract class CometNativeExec extends CometExec {
/** The Comet native operator */
def nativeOp: Operator
- override protected def doPrepare(): Unit = prepareSubqueries(originalPlan)
+ override protected def doPrepare(): Unit = prepareSubqueries(this)
override lazy val metrics: Map[String, SQLMetric] =
CometMetricNode.baselineMetrics(sparkContext)
@@ -217,12 +216,12 @@ abstract class CometNativeExec extends CometExec {
val it =
new CometExecIterator(CometExec.newIterId, inputs, serializedPlanCopy, nativeMetrics)
- setSubqueries(it.id, originalPlan)
+ setSubqueries(it.id, this)
Option(TaskContext.get()).foreach { context =>
context.addTaskCompletionListener[Unit] { _ =>
it.close()
- cleanSubqueries(it.id, originalPlan)
+ cleanSubqueries(it.id, this)
}
}
@@ -381,11 +380,13 @@ abstract class CometNativeExec extends CometExec {
*/
protected def canonicalizePlans(): CometNativeExec = {
def transform(arg: Any): AnyRef = arg match {
- case sparkPlan: SparkPlan if !sparkPlan.isInstanceOf[CometNativeExec] =>
+ case sparkPlan: SparkPlan
+ if !sparkPlan.isInstanceOf[CometNativeExec] &&
+ children.forall(_ != sparkPlan) =>
// Different to Spark, Comet native query node might have a Spark plan as Product element.
// We need to canonicalize the Spark plan. But it cannot be another Comet native query node,
// otherwise it will cause recursive canonicalization.
- sparkPlan.canonicalized
+ null
case other: AnyRef => other
case null => null
}
@@ -412,8 +413,8 @@ case class SerializedPlan(plan: Option[Array[Byte]]) {
case class CometProjectExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
- projectList: Seq[NamedExpression],
override val output: Seq[Attribute],
+ projectList: Seq[NamedExpression],
child: SparkPlan,
override val serializedPlanOpt: SerializedPlan)
extends CometUnaryExec
@@ -443,18 +444,20 @@ case class CometProjectExec(
case class CometFilterExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
condition: Expression,
child: SparkPlan,
override val serializedPlanOpt: SerializedPlan)
extends CometUnaryExec {
override def outputPartitioning: Partitioning = child.outputPartitioning
+ override def outputOrdering: Seq[SortOrder] = child.outputOrdering
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
this.copy(child = newChild)
override def stringArgs: Iterator[Any] =
- Iterator(originalPlan.output, condition, child)
+ Iterator(output, condition, child)
override def equals(obj: Any): Boolean = {
obj match {
@@ -480,6 +483,8 @@ case class CometFilterExec(
case class CometSortExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
+ override val outputOrdering: Seq[SortOrder],
sortOrder: Seq[SortOrder],
child: SparkPlan,
override val serializedPlanOpt: SerializedPlan)
@@ -491,7 +496,7 @@ case class CometSortExec(
this.copy(child = newChild)
override def stringArgs: Iterator[Any] =
- Iterator(originalPlan.output, sortOrder, child)
+ Iterator(output, sortOrder, child)
override def equals(obj: Any): Boolean = {
obj match {
@@ -520,7 +525,9 @@ case class CometLocalLimitExec(
override val serializedPlanOpt: SerializedPlan)
extends CometUnaryExec {
+ override def output: Seq[Attribute] = child.output
override def outputPartitioning: Partitioning = child.outputPartitioning
+ override def outputOrdering: Seq[SortOrder] = child.outputOrdering
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
this.copy(child = newChild)
@@ -550,7 +557,9 @@ case class CometGlobalLimitExec(
override val serializedPlanOpt: SerializedPlan)
extends CometUnaryExec {
+ override def output: Seq[Attribute] = child.output
override def outputPartitioning: Partitioning = child.outputPartitioning
+ override def outputOrdering: Seq[SortOrder] = child.outputOrdering
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
this.copy(child = newChild)
@@ -573,10 +582,13 @@ case class CometGlobalLimitExec(
case class CometExpandExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
projections: Seq[Seq[Expression]],
child: SparkPlan,
override val serializedPlanOpt: SerializedPlan)
extends CometUnaryExec {
+ override def outputPartitioning: Partitioning = UnknownPartitioning(0)
+
override def producedAttributes: AttributeSet = outputSet
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
@@ -600,7 +612,10 @@ case class CometExpandExec(
override lazy val metrics: Map[String, SQLMetric] = Map.empty
}
-case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[SparkPlan])
+case class CometUnionExec(
+ override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
+ children: Seq[SparkPlan])
extends CometExec {
override def doExecuteColumnar(): RDD[ColumnarBatch] = {
sparkContext.union(children.map(_.executeColumnar()))
@@ -634,8 +649,10 @@ case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[Sp
case class CometHashAggregateExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
groupingExpressions: Seq[NamedExpression],
aggregateExpressions: Seq[AggregateExpression],
+ resultExpressions: Seq[NamedExpression],
input: Seq[Attribute],
mode: Option[AggregateMode],
child: SparkPlan,
@@ -674,13 +691,14 @@ case class CometHashAggregateExec(
override def hashCode(): Int =
Objects.hashCode(groupingExpressions, aggregateExpressions, input, mode, child)
- override protected def outputExpressions: Seq[NamedExpression] =
- originalPlan.asInstanceOf[HashAggregateExec].resultExpressions
+ override protected def outputExpressions: Seq[NamedExpression] = resultExpressions
}
case class CometHashJoinExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
+ override val outputOrdering: Seq[SortOrder],
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
joinType: JoinType,
@@ -733,6 +751,8 @@ case class CometHashJoinExec(
case class CometBroadcastHashJoinExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
+ override val outputOrdering: Seq[SortOrder],
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
joinType: JoinType,
@@ -867,6 +887,8 @@ case class CometBroadcastHashJoinExec(
case class CometSortMergeJoinExec(
override val nativeOp: Operator,
override val originalPlan: SparkPlan,
+ override val output: Seq[Attribute],
+ override val outputOrdering: Seq[SortOrder],
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
joinType: JoinType,
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
index dc0fb18f3..d6f75837b 100644
--- 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
@@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,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]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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
@@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #4
@@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,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]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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
@@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_customer_id]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt
index 19243e359..efd4b187d 100644
--- 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
@@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_county,ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [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]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt
index dd1a52206..127a5a291 100644
--- 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
@@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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
@@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt
index 905a35c8b..4c2d3f3d7 100644
--- 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
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt
index 4de403664..98d995db0 100644
--- 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
@@ -5,18 +5,18 @@ WholeStageCodegen (2)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,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]
+ CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status]
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]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [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,s_store_sk]
+ CometFilter [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]
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
@@ -24,22 +24,22 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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
+ CometBroadcastExchange [hd_demo_sk,hd_dep_count] #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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt
index e3dfa631b..2257d398f 100644
--- 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
@@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
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]
+ 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
BroadcastExchange #5
@@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #7
@@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10
+ 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 #11
+ CometBroadcastExchange [d_date_sk] #11
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #11
InputAdapter
@@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [i_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ 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] #4
@@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [cs_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [ws_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk]
+ CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_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
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
index 799f74a36..aa49638c7 100644
--- 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
@@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #2
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
@@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ 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
BroadcastExchange #4
@@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #3
BroadcastExchange #6
@@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ 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 #10
+ CometBroadcastExchange [d_date_sk] #10
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #10
InputAdapter
@@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #6
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
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
index a03346372..b876fe4de 100644
--- 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
@@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_zip,cs_sales_price]
+ CometHashAggregate [ca_zip,sum,cs_sales_price]
CometProject [cs_sales_price,ca_zip]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,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]
+ CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip]
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]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk]
+ CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_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
@@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #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]
+ CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4
+ CometFilter [ca_address_sk,ca_state,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt
index a55c182be..8935abb54 100644
--- 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
@@ -26,7 +26,7 @@ WholeStageCodegen (12)
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]
+ CometFilter [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]
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)
@@ -54,7 +54,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -62,7 +62,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -70,5 +70,5 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [cc_call_center_sk]
- CometFilter [cc_county,cc_call_center_sk]
+ CometFilter [cc_call_center_sk,cc_county]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt
index 77aba376e..ae6bab279 100644
--- 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
@@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity]
+ CometHashAggregate [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,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc]
CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
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
@@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
ReusedExchange [d_date_sk] #7
- CometBroadcastExchange #8
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #8
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #9
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt
index 3d101857b..2519d58a2 100644
--- 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
@@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
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]
+ CometHashAggregate [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,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]
+ CometBroadcastHashJoin [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,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_county,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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
@@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3
CometProject [cd_demo_sk,cd_dep_count]
- CometFilter [cd_gender,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #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]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year]
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
+ CometBroadcastExchange [cd_demo_sk] #5
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #6
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6
+ CometFilter [ca_address_sk,ca_county,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #8
+ CometFilter [i_item_sk,i_item_id]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt
index 18a69bcb4..163a31f47 100644
--- 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
@@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price]
+ CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,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]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,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]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip]
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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact]
CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3
CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk,ca_zip] #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]
+ CometBroadcastExchange [s_store_sk,s_zip] #6
+ CometFilter [s_store_sk,s_zip]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt
index c7999d981..5d0658192 100644
--- 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
@@ -13,16 +13,16 @@ WholeStageCodegen (7)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_week_seq,d_day_name,sales_price]
+ CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price]
CometProject [sales_price,d_week_seq,d_day_name]
- CometBroadcastHashJoin [sold_date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name]
+ CometUnion [sold_date_sk,sales_price]
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]
+ CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3
+ CometFilter [d_date_sk,d_week_seq,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name]
InputAdapter
BroadcastExchange #4
@@ -30,7 +30,7 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_week_seq]
+ CometFilter [d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year]
InputAdapter
BroadcastExchange #5
@@ -46,5 +46,5 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_week_seq]
+ CometFilter [d_week_seq,d_year]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt
index d805e3868..08088a386 100644
--- 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
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt
index 52bd7a85e..bd81a3a7b 100644
--- 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
@@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,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]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date]
CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk,i_item_id] #4
CometProject [i_item_sk,i_item_id]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price]
- CometBroadcastExchange #5
- CometFilter [d_date,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_date] #5
+ CometFilter [d_date_sk,d_date]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt
index bda583c17..7d36dc400 100644
--- 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
@@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand]
+ CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,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]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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
@@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [w_warehouse_sk] #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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt
index f818fd25f..e065a0a87 100644
--- 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
@@ -28,7 +28,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -41,12 +41,12 @@ WholeStageCodegen (18)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date,count]
CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc]
CometProject [ss_item_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #6
@@ -54,14 +54,14 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #8
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
InputAdapter
WholeStageCodegen (6)
@@ -81,12 +81,12 @@ WholeStageCodegen (18)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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
@@ -94,12 +94,12 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [c_customer_sk] #10
- CometBroadcastExchange #14
+ CometBroadcastExchange [d_date_sk] #14
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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
@@ -107,13 +107,13 @@ WholeStageCodegen (18)
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk]
CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #10
+ CometBroadcastExchange [c_customer_sk] #10
CometFilter [c_customer_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk]
InputAdapter
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
index 54ee3dbde..2862f64cc 100644
--- 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
@@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
BroadcastHashJoin [cs_item_sk,item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date,count]
CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc]
CometProject [ss_item_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #6
@@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #8
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
InputAdapter
WholeStageCodegen (6)
@@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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
@@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [c_customer_sk] #10
- CometBroadcastExchange #14
+ CometBroadcastExchange [d_date_sk] #14
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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
@@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk]
CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #10
+ CometBroadcastExchange [c_customer_sk] #10
CometFilter [c_customer_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk]
InputAdapter
@@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
WholeStageCodegen (10)
@@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
BroadcastHashJoin [ws_item_sk,item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_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
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
index 8ebd45fd1..30091beb8 100644
--- 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
@@ -39,7 +39,7 @@ WholeStageCodegen (11)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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
@@ -74,7 +74,7 @@ WholeStageCodegen (11)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -85,7 +85,7 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #5
@@ -93,26 +93,26 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,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]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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]
+ CometFilter [c_customer_sk,c_first_name,c_last_name,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]
+ CometFilter [ca_state,ca_zip,ca_country]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt
index 8ebd45fd1..30091beb8 100644
--- 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
@@ -39,7 +39,7 @@ WholeStageCodegen (11)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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
@@ -74,7 +74,7 @@ WholeStageCodegen (11)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -85,7 +85,7 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #5
@@ -93,26 +93,26 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,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]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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]
+ CometFilter [c_customer_sk,c_first_name,c_last_name,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]
+ CometFilter [ca_state,ca_zip,ca_country]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt
index 74a5e8777..1f4faaac5 100644
--- 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
@@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
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]
+ CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name]
CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
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
@@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
- CometBroadcastExchange #9
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt
index cba306f68..a39fbbb79 100644
--- 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
@@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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]
+ CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,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]
+ CometBroadcastHashJoin [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,cd_demo_sk]
+ CometFilter [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]
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
@@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [p_promo_sk] #6
CometProject [p_promo_sk]
- CometFilter [p_channel_email,p_channel_event,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_email,p_channel_event]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt
index 7fcbe967a..9c4dd659a 100644
--- 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
@@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price]
+ CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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
@@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [s_state,s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #5
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt
index 47a4fe2c7..80c886708 100644
--- 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
@@ -15,9 +15,9 @@ WholeStageCodegen (18)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -33,9 +33,9 @@ WholeStageCodegen (18)
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -51,9 +51,9 @@ WholeStageCodegen (18)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -69,9 +69,9 @@ WholeStageCodegen (18)
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -87,9 +87,9 @@ WholeStageCodegen (18)
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -105,7 +105,7 @@ WholeStageCodegen (18)
WholeStageCodegen (15)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt
index 2bcef6168..88b4ecd1d 100644
--- 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
@@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity]
+ CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc]
CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
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
@@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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
@@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #8
+ CometBroadcastExchange [d_date_sk] #8
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk] #9
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #10
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
- CometBroadcastExchange #11
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt
index cc9c4edf3..bccda58a4 100644
--- 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
@@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price]
+ CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price]
CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt
index f24e34c36..a80e0ca71 100644
--- 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
@@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt]
+ CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt]
CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
- CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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
@@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
@@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt]
+ CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt]
CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
- CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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
@@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [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]
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
@@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt
index a94a8a94d..a48c7ad26 100644
--- 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
@@ -17,24 +17,24 @@ WholeStageCodegen (13)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk,ca_county] #5
CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
InputAdapter
@@ -46,22 +46,22 @@ WholeStageCodegen (13)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #9
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
ReusedExchange [ca_address_sk,ca_county] #5
InputAdapter
@@ -73,22 +73,22 @@ WholeStageCodegen (13)
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #13
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
ReusedExchange [ca_address_sk,ca_county] #5
InputAdapter
@@ -100,12 +100,12 @@ WholeStageCodegen (13)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
@@ -119,12 +119,12 @@ WholeStageCodegen (13)
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
@@ -138,12 +138,12 @@ WholeStageCodegen (13)
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
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
index b8df1e929..c3bfdfd77 100644
--- 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
@@ -11,8 +11,8 @@ WholeStageCodegen (5)
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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk]
+ CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk]
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
@@ -20,11 +20,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk] #3
CometProject [i_item_sk]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
InputAdapter
BroadcastExchange #4
@@ -36,15 +36,15 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_item_sk,cs_ext_discount_amt]
+ CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt]
CometProject [cs_item_sk,cs_ext_discount_amt]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt
index af2b7cb5d..7a3c2c153 100644
--- 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
@@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,ss_ext_sales_price]
+ CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
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]
+ CometBroadcastExchange [i_item_sk,i_manufact_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_manufact_id] #7
CometProject [i_manufact_id]
- CometFilter [i_category]
+ CometFilter [i_category,i_manufact_id]
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]
@@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,cs_ext_sales_price]
+ CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,ws_ext_sales_price]
+ CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
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
index eefd38343..e1f7f9bd2 100644
--- 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
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt
index efe0b0b4e..9deb51342 100644
--- 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
@@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt
index 574a20fc0..4ad9b501b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt
@@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price]
+ CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price]
CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id]
CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category]
CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #5
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt
index c3fcd79f0..efd86d5ab 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt
@@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
InputAdapter
CometHashAggregate [i_item_id,i_item_desc,i_current_price]
CometProject [i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometBroadcastExchange #2
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk]
- CometBroadcastHashJoin [i_item_sk,inv_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometFilter [i_current_price,i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
- CometBroadcastExchange #3
+ CometBroadcastExchange [inv_item_sk,inv_date_sk] #3
CometProject [inv_item_sk,inv_date_sk]
- CometFilter [inv_quantity_on_hand,inv_item_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [cs_item_sk]
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt
index cfac83844..9d667265c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt
@@ -15,10 +15,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ss_customer_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,14 +26,14 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_date] #4
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #6
@@ -46,10 +46,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [cs_bill_customer_sk,d_date]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_bill_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date]
+ CometFilter [cs_bill_customer_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #4
@@ -65,10 +65,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ws_bill_customer_sk,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt
index 7d28d5c10..0c709e4f7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #3
@@ -27,17 +27,17 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk] #4
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_moy] #6
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #7
@@ -50,14 +50,14 @@ WholeStageCodegen (5)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -65,11 +65,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
ReusedExchange [i_item_sk] #4
ReusedExchange [w_warehouse_sk,w_warehouse_name] #5
- CometBroadcastExchange #10
+ CometBroadcastExchange [d_date_sk,d_moy] #10
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt
index 7d28d5c10..0c709e4f7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #3
@@ -27,17 +27,17 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk] #4
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_moy] #6
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #7
@@ -50,14 +50,14 @@ WholeStageCodegen (5)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -65,11 +65,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
ReusedExchange [i_item_sk] #4
ReusedExchange [w_warehouse_sk,w_warehouse_name] #5
- CometBroadcastExchange #10
+ CometBroadcastExchange [d_date_sk,d_moy] #10
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt
index 048da153b..1a19c58ff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt
@@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (1)
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,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (2)
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,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 [cs_bill_customer_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12
+ CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 [cs_bill_customer_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15
+ CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [d_date_sk,d_year] #9
@@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (8)
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_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 #18
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (10)
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_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 #21
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,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/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt
index 10e0735b4..11c0201a8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt
@@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [cs_warehouse_sk,cs_item_sk]
+ CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [w_warehouse_sk]
+ CometFilter [w_warehouse_sk,w_state]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state]
InputAdapter
BroadcastExchange #6
@@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_item_id]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price]
InputAdapter
ReusedExchange [d_date_sk,d_date] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt
index e31217066..457d441ce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt
@@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name]
ColumnarToRow
InputAdapter
CometProject [i_manufact,i_product_name]
- CometFilter [i_manufact_id,i_manufact]
+ CometFilter [i_manufact_id,i_manufact,i_product_name]
CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name]
InputAdapter
BroadcastExchange #2
@@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact]
+ CometHashAggregate [i_manufact,count]
CometProject [i_manufact]
- CometFilter [i_category,i_color,i_units,i_size,i_manufact]
+ CometFilter [i_category,i_manufact,i_size,i_color,i_units]
CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt
index 3e69a3341..e3b850ef5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price]
+ CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price]
CometProject [d_year,ss_ext_sales_price,i_category_id,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3
CometProject [i_item_sk,i_category_id,i_category]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_category_id,i_category,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt
index d29a65bd4..5a060b85c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price]
+ CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price]
CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name]
CometProject [d_day_name,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_day_name]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name]
- CometBroadcastExchange #2
- CometFilter [ss_store_sk]
+ CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3
CometProject [s_store_sk,s_store_id,s_store_name]
- CometFilter [s_gmt_offset,s_store_sk]
+ CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt
index 0a66507e1..232d65d72 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt
@@ -30,9 +30,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_store_sk,ss_net_profit]
+ CometHashAggregate [ss_store_sk,sum,count,ss_net_profit]
CometProject [ss_store_sk,ss_net_profit]
- CometFilter [ss_store_sk,ss_addr_sk]
+ CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
InputAdapter
@@ -40,9 +40,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,ss_net_profit]
+ CometHashAggregate [ss_item_sk,sum,count,ss_net_profit]
CometProject [ss_item_sk,ss_net_profit]
- CometFilter [ss_store_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -69,7 +69,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name]
InputAdapter
ReusedExchange [i_item_sk,i_product_name] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt
index c376c4fcd..f60fdb18a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt
@@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [ws_sales_price,ca_city,ca_zip,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk]
CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip]
CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
- CometFilter [ws_bill_customer_sk,ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #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]
+ CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4
+ CometFilter [ca_address_sk,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
InputAdapter
BroadcastExchange #7
@@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [i_item_id]
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_item_id]
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/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt
index 93e3eb05b..57defd961 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt
@@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_city,s_store_sk]
+ CometFilter [s_store_sk,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk,ca_city] #6
CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
InputAdapter
@@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt
index d2e615f39..b6e5c469b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price]
+ CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price]
CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt
index 6ebf6af07..5f628dbae 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt
@@ -5,16 +5,16 @@ WholeStageCodegen (2)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_quantity]
+ CometHashAggregate [sum,ss_quantity]
CometProject [ss_quantity]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk]
CometProject [ss_quantity,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit]
+ CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state]
CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -22,19 +22,19 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4
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 #5
+ CometBroadcastExchange [ca_address_sk,ca_state] #5
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt
index 8d7b158d0..0e6b65b06 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
+ CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
- CometBroadcastExchange #4
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
+ CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk]
- CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_return_amt,wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (10)
Project [item,return_ratio,return_rank,currency_rank]
@@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
+ CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
+ CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk]
- CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_return_amount,cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
ReusedExchange [d_date_sk] #6
WholeStageCodegen (15)
@@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
+ CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
- CometBroadcastExchange #12
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
+ CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk]
- CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
ReusedExchange [d_date_sk] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt
index c4ea8fe24..537cba446 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt
@@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,s_store_id]
- CometBroadcastHashJoin [store_sk,s_store_sk]
+ CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id]
CometProject [store_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ss_store_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [sr_store_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
WholeStageCodegen (4)
HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum]
@@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id]
- CometBroadcastHashJoin [page_sk,cp_catalog_page_sk]
+ CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id]
CometProject [page_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cs_catalog_page_sk]
+ CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cr_catalog_page_sk]
+ CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #7
- CometFilter [cp_catalog_page_sk]
+ CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
WholeStageCodegen (6)
HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum]
@@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,web_site_id]
- CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id]
CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ws_web_site_sk]
+ CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number]
+ CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_item_sk,ws_web_site_sk,ws_order_number]
- CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #10
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_site_id] #10
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt
index 473b9cdd4..58adabcec 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt
@@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk]
+ CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk]
- CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4
+ CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometFilter [d_date_sk]
CometScan parquet spark_catalog.default.date_dim [d_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt
index ce6005da6..988297f02 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt
@@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,d_date,ws_sales_price]
+ CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price]
CometProject [ws_item_sk,ws_sales_price,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_date] #6
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
InputAdapter
WholeStageCodegen (10)
@@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,d_date,ss_sales_price]
+ CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price]
CometProject [ss_item_sk,ss_sales_price,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt
index 557dd3b4a..a0932c7cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price]
+ CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price]
CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt
index 83e53bb66..b2199fb34 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt
@@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price]
+ CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price]
CometProject [i_manufact_id,ss_sales_price,d_qoy]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk]
CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy]
CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [i_item_sk,i_manufact_id]
- CometFilter [i_category,i_class,i_brand,i_item_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_qoy] #5
CometProject [d_date_sk,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt
index c6886735f..2193d875c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt
@@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [c_customer_sk,c_current_addr_sk]
CometProject [c_customer_sk,c_current_addr_sk]
- CometBroadcastHashJoin [customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk]
CometProject [customer_sk]
- CometBroadcastHashJoin [sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk]
CometProject [sold_date_sk,customer_sk]
- CometBroadcastHashJoin [item_sk,i_item_sk]
- CometUnion
+ CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk]
+ CometUnion [sold_date_sk,customer_sk,item_sk]
CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk]
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk]
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #5
+ CometBroadcastExchange [i_item_sk] #5
CometProject [i_item_sk]
- CometFilter [i_category,i_class,i_item_sk]
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #7
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
InputAdapter
@@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
Subquery #3
WholeStageCodegen (2)
HashAggregate [(d_month_seq + 1)]
@@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [(d_month_seq + 1)]
CometProject [d_month_seq] [(d_month_seq + 1)]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
Subquery #4
WholeStageCodegen (2)
@@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [(d_month_seq + 3)]
CometProject [d_month_seq] [(d_month_seq + 3)]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt
index 2750a6ba2..fc38884c1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price]
+ CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt
index 7fdead831..0b48046fd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt
@@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #6
- CometBroadcastHashJoin [i_item_id,i_item_id]
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id]
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_item_id] #7
CometProject [i_item_id]
- CometFilter [i_color]
+ CometFilter [i_item_id,i_color]
CometScan parquet spark_catalog.default.item [i_item_id,i_color]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum]
@@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt
index c630cad48..fb2b09b9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price]
+ CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price]
CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [cs_item_sk,cs_call_center_sk]
+ CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3
+ CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #6
CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt
index 2ed2bde44..fc7c21e94 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt
@@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date] #3
CometProject [d_date]
- CometFilter [d_week_seq]
+ CometFilter [d_date,d_week_seq]
Subquery #2
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_date]
+ CometFilter [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk,i_item_id] #4
CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
ReusedExchange [d_date] #3
InputAdapter
@@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk]
CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #4
@@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk]
CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt
index e00d52dbb..6c1e6cd92 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt
@@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price]
+ CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price]
CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name]
+ CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [d_date_sk,d_week_seq]
+ CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2
+ CometFilter [d_date_sk,d_week_seq,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk,s_store_id]
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
InputAdapter
BroadcastExchange #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt
index 89a080d85..824670101 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt
@@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [ca_state,ss_item_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk]
CometProject [ca_state,ss_item_sk,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometProject [ca_state,c_customer_sk]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #2
- CometFilter [c_current_addr_sk,c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2
+ CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #3
- CometFilter [ss_customer_sk,ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
Subquery #2
WholeStageCodegen (2)
HashAggregate [d_month_seq]
@@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state]
InputAdapter
CometHashAggregate [d_month_seq]
CometProject [d_month_seq]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
@@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state]
BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
ColumnarToRow
InputAdapter
- CometFilter [i_current_price,i_category,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category]
InputAdapter
BroadcastExchange #8
@@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_current_price]
- CometFilter [i_category]
+ CometHashAggregate [i_category,sum,count,i_current_price]
+ CometFilter [i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_current_price,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt
index b76e7c9b3..7cfcb75da 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt
@@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #6
- CometBroadcastHashJoin [i_item_id,i_item_id]
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id]
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_item_id] #7
CometProject [i_item_id]
- CometFilter [i_category]
+ CometFilter [i_item_id,i_category]
CometScan parquet spark_catalog.default.item [i_item_id,i_category]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum]
@@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt
index 4ca1dd667..1b2af33b0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt
@@ -7,20 +7,20 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_sales_price]
+ CometHashAggregate [sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk]
CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -28,30 +28,30 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometProject [s_store_sk]
- CometFilter [s_gmt_offset,s_store_sk]
+ CometFilter [s_store_sk,s_gmt_offset]
CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset]
- CometBroadcastExchange #4
+ CometBroadcastExchange [p_promo_sk] #4
CometProject [p_promo_sk]
- CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #7
+ CometBroadcastExchange [ca_address_sk] #7
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #8
+ CometBroadcastExchange [i_item_sk] #8
CometProject [i_item_sk]
- CometFilter [i_category,i_item_sk]
+ CometFilter [i_item_sk,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category]
InputAdapter
BroadcastExchange #9
@@ -62,18 +62,18 @@ WholeStageCodegen (4)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_sales_price]
+ CometHashAggregate [sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk]
CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [s_store_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt
index c6b7e1834..0b4fc61a1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt
@@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk]
+ CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk]
CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression]
- CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk]
CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name]
- CometBroadcastHashJoin [ws_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name]
CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type]
- CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type]
CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk]
- CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #3
- CometFilter [sm_ship_mode_sk]
+ CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3
+ CometFilter [sm_ship_mode_sk,sm_type]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type]
- CometBroadcastExchange #4
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_name] #4
+ CometFilter [web_site_sk,web_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt
index 35e09ec7d..b2033c7a9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt
@@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manager_id,d_moy,ss_sales_price]
+ CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price]
CometProject [i_manager_id,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk]
CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy]
CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [i_item_sk,i_manager_id]
- CometFilter [i_category,i_class,i_brand,i_item_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_moy] #5
CometProject [d_date_sk,d_moy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt
index d972e0082..3b5e4f19e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt
@@ -54,19 +54,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #4
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -89,7 +89,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_ext_list_price]
- CometFilter [cs_item_sk,cs_order_number]
+ CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -100,7 +100,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #5
@@ -116,14 +116,14 @@ WholeStageCodegen (52)
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometFilter [d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
ReusedExchange [d_date_sk,d_year] #11
@@ -157,7 +157,7 @@ WholeStageCodegen (52)
WholeStageCodegen (19)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
InputAdapter
ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
@@ -176,7 +176,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_product_name]
- CometFilter [i_current_price,i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_color,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name]
InputAdapter
WholeStageCodegen (50)
@@ -227,19 +227,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #20
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #21
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (33)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt
index 8de564ed1..6cb247973 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
BroadcastHashJoin [s_store_sk,ss_store_sk]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
InputAdapter
BroadcastExchange #1
@@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price]
+ CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price]
CometProject [ss_item_sk,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand]
InputAdapter
BroadcastExchange #6
@@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price]
+ CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price]
CometProject [ss_item_sk,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt
index d746739b3..72133811b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt
@@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid]
CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk]
CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk]
CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy]
CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk]
- CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
+ CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4
+ CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastExchange #5
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [t_time_sk] #6
CometProject [t_time_sk]
- CometFilter [t_time,t_time_sk]
+ CometFilter [t_time_sk,t_time]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time]
- CometBroadcastExchange #7
+ CometBroadcastExchange [sm_ship_mode_sk] #7
CometProject [sm_ship_mode_sk]
- CometFilter [sm_carrier,sm_ship_mode_sk]
+ CometFilter [sm_ship_mode_sk,sm_carrier]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier]
WholeStageCodegen (4)
HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
@@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax]
CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk]
CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk]
CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy]
CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk]
- CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
+ CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt
index ef965d7d0..4dc4bcaf5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt
@@ -17,15 +17,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity]
+ CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity]
CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id]
CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy]
+ CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt
index 43f44c9f7..c5f56a69c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt
@@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_city,s_store_sk]
+ CometFilter [s_store_sk,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk,ca_city] #6
CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
InputAdapter
@@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt
index de2d5eeda..c9cc4959b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt
index c583ba8e8..f327d5b06 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt
@@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price]
CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id]
- CometBroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk]
CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_promo_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_item_sk,ss_promo_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [p_promo_sk] #6
CometProject [p_promo_sk]
- CometFilter [p_channel_email,p_channel_event,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_email,p_channel_event]
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/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt
index 7cddbc640..030dfc4d8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt
@@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
BroadcastHashJoin [s_state,s_state]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_county,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state]
InputAdapter
BroadcastExchange #6
@@ -58,15 +58,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_state,ss_net_profit]
+ CometHashAggregate [s_state,sum,ss_net_profit]
CometProject [ss_net_profit,s_state]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk]
CometProject [ss_net_profit,ss_sold_date_sk,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #8
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #8
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt
index fd6777886..c604a8fa7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt
@@ -9,19 +9,19 @@ WholeStageCodegen (3)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price]
+ CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price]
CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute]
- CometBroadcastHashJoin [time_sk,t_time_sk]
+ CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute]
CometProject [i_brand_id,i_brand,ext_price,time_sk]
- CometBroadcastHashJoin [i_item_sk,sold_item_sk]
- CometBroadcastExchange #3
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
- CometUnion
+ CometUnion [ext_price,sold_item_sk,time_sk]
CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk,ws_sold_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -29,25 +29,25 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_item_sk,cs_sold_time_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_sold_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #6
+ CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6
CometProject [t_time_sk,t_hour,t_minute]
- CometFilter [t_meal_time,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute,t_meal_time]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt
index bea1fd4a1..6cba2d0e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt
@@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk]
- CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity]
- CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #4
- CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
+ CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #6
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
- CometBroadcastExchange #7
+ CometBroadcastExchange [cd_demo_sk] #7
CometProject [cd_demo_sk]
- CometFilter [cd_marital_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_marital_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status]
- CometBroadcastExchange #8
+ CometBroadcastExchange [hd_demo_sk] #8
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #10
- CometFilter [d_week_seq,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_week_seq] #10
+ CometFilter [d_date_sk,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
@@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt
index 060c3e153..c91c4cf47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt
index 2e6286d93..7c2a42ca3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt
@@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #2
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #7
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #12
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #15
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,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/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt
index 34866bc37..27a4dcb67 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt
@@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
- CometBroadcastExchange #7
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #7
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
WholeStageCodegen (10)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
@@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
WholeStageCodegen (15)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
@@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (38)
@@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #17
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
- CometBroadcastExchange #18
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #18
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (23)
@@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt
index 473eef9e4..e463296d4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt
@@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometUnion
+ CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price]
+ CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_category] #2
+ CometFilter [i_item_sk,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category]
- CometBroadcastExchange #3
- CometFilter [d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_ship_customer_sk,ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category]
+ CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk]
ReusedExchange [i_item_sk,i_category] #2
ReusedExchange [d_date_sk,d_year,d_qoy] #3
CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_ship_addr_sk,cs_item_sk]
+ CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category]
+ CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedExchange [i_item_sk,i_category] #2
ReusedExchange [d_date_sk,d_year,d_qoy] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt
index 590c59fdc..f9088f784 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt
@@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit]
+ CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit]
CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
InputAdapter
@@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss]
+ CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss]
CometProject [sr_return_amt,sr_net_loss,s_store_sk]
- CometBroadcastHashJoin [sr_store_sk,s_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk]
CometProject [sr_store_sk,sr_return_amt,sr_net_loss]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- CometFilter [sr_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
+ CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit]
CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_return_amount,cr_net_loss]
+ CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss]
CometProject [cr_return_amount,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit]
+ CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit]
CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #12
+ CometBroadcastExchange [wp_web_page_sk] #12
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
InputAdapter
@@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss]
+ CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss]
CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk]
- CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk]
CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
- CometFilter [wr_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt
index 280687e30..13a8aecff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk,ss_customer_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt
index b68a9474b..57598ec35 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt
@@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1,
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1,
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk,s_city] #4
CometProject [s_store_sk,s_city]
- CometFilter [s_number_employees,s_store_sk]
+ CometFilter [s_store_sk,s_number_employees,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
InputAdapter
BroadcastExchange #6
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt
index adf555417..637f3b7ba 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt
@@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
ColumnarToRow
InputAdapter
CometProject [ss_net_profit,s_store_name,s_zip]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip]
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
- CometFilter [s_store_sk,s_zip]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4
+ CometFilter [s_store_sk,s_store_name,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip]
InputAdapter
BroadcastExchange #5
@@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_zip]
+ CometHashAggregate [ca_zip,count]
CometProject [ca_zip]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip]
- CometBroadcastExchange #9
+ CometBroadcastExchange [c_current_addr_sk] #9
CometProject [c_current_addr_sk]
- CometFilter [c_preferred_cust_flag,c_current_addr_sk]
+ CometFilter [c_current_addr_sk,c_preferred_cust_flag]
CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt
index 7e257bdc6..cdc8dc486 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt
@@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
InputAdapter
BroadcastExchange #7
@@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [i_item_sk]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price]
InputAdapter
BroadcastExchange #8
@@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [p_promo_sk]
- CometFilter [p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv]
WholeStageCodegen (20)
HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
@@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk]
+ CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cp_catalog_page_sk]
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
InputAdapter
ReusedExchange [i_item_sk] #7
@@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (21)
ColumnarToRow
InputAdapter
- CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss]
- CometFilter [wr_item_sk,wr_order_number]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (26)
ColumnarToRow
InputAdapter
- CometFilter [web_site_sk]
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
InputAdapter
ReusedExchange [i_item_sk] #7
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt
index ac7b2c7e0..11f3a0ff1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt
@@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax]
+ CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax]
CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state]
- CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state]
CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
- CometFilter [cr_returning_addr_sk,cr_returning_customer_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk]
+ CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
@@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax]
+ CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax]
CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state]
- CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state]
CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
- CometFilter [cr_returning_addr_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk]
+ CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #3
@@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt
index a4d96e6f1..71a1c8e17 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt
@@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
InputAdapter
CometHashAggregate [i_item_id,i_item_desc,i_current_price]
CometProject [i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometBroadcastExchange #2
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk]
- CometBroadcastHashJoin [i_item_sk,inv_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometFilter [i_current_price,i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
- CometBroadcastExchange #3
+ CometBroadcastExchange [inv_item_sk,inv_date_sk] #3
CometProject [inv_item_sk,inv_date_sk]
- CometFilter [inv_quantity_on_hand,inv_item_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [ss_item_sk]
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt
index ef7d35e21..390c20b7f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt
@@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,sr_return_quantity]
+ CometHashAggregate [i_item_id,sum,sr_return_quantity]
CometProject [sr_return_quantity,i_item_id]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk]
CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [sr_item_sk,i_item_sk]
- CometFilter [sr_item_sk]
+ CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date] #3
CometProject [d_date]
- CometBroadcastHashJoin [d_week_seq,d_week_seq]
+ CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_week_seq] #4
CometProject [d_week_seq]
- CometFilter [d_date]
+ CometFilter [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
ReusedExchange [d_date] #3
InputAdapter
@@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cr_return_quantity]
+ CometHashAggregate [i_item_id,sum,cr_return_quantity]
CometProject [cr_return_quantity,i_item_id]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk]
CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [cr_item_sk,i_item_sk]
- CometFilter [cr_item_sk]
+ CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #5
@@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,wr_return_quantity]
+ CometHashAggregate [i_item_id,sum,wr_return_quantity]
CometProject [wr_return_quantity,i_item_id]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk]
CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [wr_item_sk,i_item_sk]
- CometFilter [wr_item_sk]
+ CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt
index fe7f7a207..646285a08 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt
@@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername]
Project [c_customer_id,c_last_name,c_first_name]
ColumnarToRow
InputAdapter
- CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk]
- CometBroadcastExchange #1
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk]
+ CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1
CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk]
- CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk]
CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk]
- CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk]
CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk]
CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk]
+ CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ca_address_sk] #2
CometProject [ca_address_sk]
- CometFilter [ca_city,ca_address_sk]
+ CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4
CometFilter [hd_demo_sk,hd_income_band_sk]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ib_income_band_sk] #5
CometProject [ib_income_band_sk]
- CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk]
+ CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound]
CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound]
CometProject [sr_cdemo_sk]
- CometFilter [sr_cdemo_sk]
+ CometFilter [sr_cdemo_sk,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt
index ecae29c78..2313467ae 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt
@@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee]
+ CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee]
CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc]
- CometBroadcastHashJoin [wr_reason_sk,r_reason_sk]
+ CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc]
CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk]
CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit]
+ CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state]
CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status]
+ CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price]
+ CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number]
- CometBroadcastExchange #2
- CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit]
+ CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash]
+ CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2
+ CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash]
- CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk]
+ CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [wp_web_page_sk] #4
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5
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 #6
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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
+ CometBroadcastExchange [ca_address_sk,ca_state] #7
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #8
+ CometBroadcastExchange [d_date_sk] #8
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [r_reason_sk]
+ CometBroadcastExchange [r_reason_sk,r_reason_desc] #9
+ CometFilter [r_reason_sk,r_reason_desc]
CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt
index 4218938c1..a6d1c25fc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt
@@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid]
+ CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid]
CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id]
CometProject [ws_net_paid,i_category,i_class]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category]
CometProject [ws_item_sk,ws_net_paid]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #5
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt
index cfac83844..9d667265c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt
@@ -15,10 +15,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ss_customer_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,14 +26,14 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_date] #4
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #6
@@ -46,10 +46,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [cs_bill_customer_sk,d_date]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_bill_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date]
+ CometFilter [cs_bill_customer_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #4
@@ -65,10 +65,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ws_bill_customer_sk,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt
index b846d25d8..d3af10d19 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt
@@ -12,27 +12,27 @@ WholeStageCodegen (16)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_store_name,s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
InputAdapter
BroadcastExchange #5
@@ -43,20 +43,20 @@ WholeStageCodegen (16)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #7
+ CometBroadcastExchange [t_time_sk] #7
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -68,20 +68,20 @@ WholeStageCodegen (16)
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #10
+ CometBroadcastExchange [t_time_sk] #10
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -93,20 +93,20 @@ WholeStageCodegen (16)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #13
+ CometBroadcastExchange [t_time_sk] #13
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -118,20 +118,20 @@ WholeStageCodegen (16)
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #16
+ CometBroadcastExchange [t_time_sk] #16
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -143,20 +143,20 @@ WholeStageCodegen (16)
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #19
+ CometBroadcastExchange [t_time_sk] #19
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -168,20 +168,20 @@ WholeStageCodegen (16)
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #22
+ CometBroadcastExchange [t_time_sk] #22
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -193,19 +193,19 @@ WholeStageCodegen (16)
WholeStageCodegen (14)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #25
+ CometBroadcastExchange [t_time_sk] #25
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt
index 507ac8a91..aa858c158 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt
@@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price]
+ CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price]
CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy]
CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_category,i_class,i_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_moy] #5
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
+ CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt
index c54606f6e..5cb600551 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt
@@ -9,9 +9,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #1
ReusedSubquery [mergedValue] #1
@@ -24,9 +24,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #2
ReusedSubquery [mergedValue] #2
@@ -39,9 +39,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #3
ReusedSubquery [mergedValue] #3
@@ -54,9 +54,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #4
ReusedSubquery [mergedValue] #4
@@ -69,9 +69,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #5
ReusedSubquery [mergedValue] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt
index 50c8494fb..44159cc73 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt
@@ -7,27 +7,27 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
CometProject [ws_web_page_sk]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk]
CometProject [ws_sold_time_sk,ws_web_page_sk]
- CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk]
CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
- CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
+ CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_time_sk]
+ CometFilter [t_time_sk,t_hour]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour]
- CometBroadcastExchange #4
+ CometBroadcastExchange [wp_web_page_sk] #4
CometProject [wp_web_page_sk]
- CometFilter [wp_char_count,wp_web_page_sk]
+ CometFilter [wp_web_page_sk,wp_char_count]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count]
InputAdapter
BroadcastExchange #5
@@ -38,19 +38,19 @@ WholeStageCodegen (4)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
CometProject [ws_web_page_sk]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk]
CometProject [ws_sold_time_sk,ws_web_page_sk]
- CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk]
CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
- CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
+ CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #7
+ CometBroadcastExchange [t_time_sk] #7
CometProject [t_time_sk]
- CometFilter [t_hour,t_time_sk]
+ CometFilter [t_time_sk,t_hour]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour]
ReusedExchange [wp_web_page_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt
index b415eb5c4..232c174cd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt
@@ -9,23 +9,23 @@ WholeStageCodegen (3)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss]
+ CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk]
- CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk]
- CometFilter [cc_call_center_sk]
+ CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
+ CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager]
- CometBroadcastExchange #3
- CometFilter [cr_call_center_sk,cr_returning_customer_sk]
+ CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3
+ CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -33,23 +33,23 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
- CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
- CometBroadcastExchange #7
+ CometBroadcastExchange [ca_address_sk] #7
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #8
- CometFilter [cd_marital_status,cd_education_status,cd_demo_sk]
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8
+ 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 #9
+ CometBroadcastExchange [hd_demo_sk] #9
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt
index e2f498028..0681a64bd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt
@@ -11,8 +11,8 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk,ws_ext_discount_amt]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk]
+ CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -20,11 +20,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk] #3
CometProject [i_item_sk]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
InputAdapter
BroadcastExchange #4
@@ -36,15 +36,15 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,ws_ext_discount_amt]
+ CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt]
CometProject [ws_item_sk,ws_ext_discount_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt
index 3ec7ac7b6..a8eb231b7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt
@@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity]
- CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk]
+ CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
InputAdapter
BroadcastExchange #4
@@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk]
ColumnarToRow
InputAdapter
CometProject [r_reason_sk]
- CometFilter [r_reason_desc,r_reason_sk]
+ CometFilter [r_reason_sk,r_reason_desc]
CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt
index 34ddde768..d54c9e0c9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt
@@ -26,7 +26,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit]
- CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk]
+ CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (4)
@@ -54,7 +54,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -62,7 +62,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -70,5 +70,5 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [web_site_sk]
- CometFilter [web_company_name,web_site_sk]
+ CometFilter [web_site_sk,web_company_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt
index 5b699890c..178c25f66 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt
@@ -25,7 +25,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit]
- CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk]
+ CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (7)
@@ -40,7 +40,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ws_warehouse_sk,ws_order_number]
- CometFilter [ws_order_number,ws_warehouse_sk]
+ CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -60,7 +60,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [wr_order_number]
- CometFilter [wr_order_number]
+ CometFilter [wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (15)
@@ -82,7 +82,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -90,7 +90,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -98,5 +98,5 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [web_site_sk]
- CometFilter [web_company_name,web_site_sk]
+ CometFilter [web_site_sk,web_company_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt
index 614915226..e9d33a7f6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt
@@ -5,25 +5,25 @@ WholeStageCodegen (2)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_store_name,s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt
index 24e6dceef..41f3e579f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt
@@ -17,7 +17,7 @@ WholeStageCodegen (6)
InputAdapter
CometHashAggregate [ss_customer_sk,ss_item_sk]
CometProject [ss_item_sk,ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -25,11 +25,11 @@ WholeStageCodegen (6)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
WholeStageCodegen (4)
@@ -42,7 +42,7 @@ WholeStageCodegen (6)
InputAdapter
CometHashAggregate [cs_bill_customer_sk,cs_item_sk]
CometProject [cs_bill_customer_sk,cs_item_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt
index c03b8be9a..3895cdc78 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt
@@ -18,12 +18,12 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -31,12 +31,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt
index adfe90bab..fc63929bb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt
@@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk]
+ CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk]
CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression]
- CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk]
CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name]
CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type]
- CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type]
CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk]
- CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #3
- CometFilter [sm_ship_mode_sk]
+ CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3
+ CometFilter [sm_ship_mode_sk,sm_type]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type]
- CometBroadcastExchange #4
- CometFilter [cc_call_center_sk]
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #4
+ CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt
index 26b5570b7..43bad5d70 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt
@@ -15,8 +15,8 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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
@@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #4
@@ -47,8 +47,8 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
CometProject [sr_customer_sk,sr_store_sk,sr_return_amt]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- CometFilter [sr_store_sk]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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
@@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_customer_id]
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-spark4_0/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt
index 19243e359..efd4b187d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt
@@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_county,ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [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]
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-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt
index fe6b3a625..240d88bb7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt
@@ -15,23 +15,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -44,23 +44,23 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -74,13 +74,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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
@@ -95,13 +95,13 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt
index 4bdb7ae0f..1640fa3b0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt
@@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt
index ef54c7add..cc52bb323 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt
@@ -7,16 +7,16 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status]
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]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [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,s_store_sk]
+ CometFilter [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]
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
@@ -24,22 +24,22 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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
+ CometBroadcastExchange [hd_demo_sk,hd_dep_count] #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-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt
index 72aa4a054..d2ea2d830 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt
@@ -20,19 +20,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
HashAggregate [quantity,list_price] [sum,count,sum,count]
ColumnarToRow
InputAdapter
- CometUnion
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
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]
+ 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
BroadcastExchange #5
@@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #7
@@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10
+ 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 #11
+ CometBroadcastExchange [d_date_sk] #11
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #11
InputAdapter
@@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [i_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ 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] #4
@@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [cs_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [ws_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk]
+ CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt
index 73badab8d..cd4817202 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt
@@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
HashAggregate [quantity,list_price] [sum,count,sum,count]
ColumnarToRow
InputAdapter
- CometUnion
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
ReusedSubquery [d_week_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
Subquery #2
@@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,d_year,d_moy,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
InputAdapter
BroadcastExchange #3
@@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ 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
BroadcastExchange #4
@@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #3
BroadcastExchange #6
@@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ 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 #10
+ CometBroadcastExchange [d_date_sk] #10
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #10
InputAdapter
@@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
ReusedSubquery [d_week_seq] #6
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
Subquery #6
@@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,d_year,d_moy,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
InputAdapter
ReusedExchange [ss_item_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt
index b0fd9ebf1..a34cd9c1b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt
@@ -8,12 +8,12 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)]
ColumnarToRow
InputAdapter
CometProject [cs_sales_price,ca_zip]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,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]
+ CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip]
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]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk]
+ CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_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
@@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #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]
+ CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4
+ CometFilter [ca_address_sk,ca_state,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
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-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt
index a55c182be..8935abb54 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt
@@ -26,7 +26,7 @@ WholeStageCodegen (12)
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]
+ CometFilter [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]
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)
@@ -54,7 +54,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -62,7 +62,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -70,5 +70,5 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [cc_call_center_sk]
- CometFilter [cc_county,cc_call_center_sk]
+ CometFilter [cc_call_center_sk,cc_county]
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-spark4_0/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt
index 5c9726a22..4a634b069 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt
@@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
ColumnarToRow
InputAdapter
CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc]
CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
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
@@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
ReusedExchange [d_date_sk] #7
- CometBroadcastExchange #8
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #8
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #9
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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-spark4_0/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt
index 561ad3d4f..c49bfc6c8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt
@@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
InputAdapter
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]
+ CometBroadcastHashJoin [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,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_county,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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
@@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3
CometProject [cd_demo_sk,cd_dep_count]
- CometFilter [cd_gender,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #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]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year]
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
+ CometBroadcastExchange [cd_demo_sk] #5
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #6
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6
+ CometFilter [ca_address_sk,ca_county,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #8
+ CometFilter [i_item_sk,i_item_id]
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-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt
index 852b61d1c..60caecf91 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt
@@ -8,31 +8,31 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact]
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,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]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip]
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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact]
CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3
CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk,ca_zip] #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]
+ CometBroadcastExchange [s_store_sk,s_zip] #6
+ CometFilter [s_store_sk,s_zip]
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-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt
index 334c6f0ea..535c61622 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt
@@ -15,14 +15,14 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [sales_price,d_week_seq,d_day_name]
- CometBroadcastHashJoin [sold_date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name]
+ CometUnion [sold_date_sk,sales_price]
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]
+ CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3
+ CometFilter [d_date_sk,d_week_seq,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name]
InputAdapter
BroadcastExchange #4
@@ -30,7 +30,7 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_week_seq]
+ CometFilter [d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year]
InputAdapter
BroadcastExchange #5
@@ -46,5 +46,5 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_week_seq]
+ CometFilter [d_week_seq,d_year]
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-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt
index dd1edb95f..5bfd9a907 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt
@@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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-spark4_0/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt
index c6058d18f..2c8892b68 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt
@@ -9,27 +9,27 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after]
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date]
CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk,i_item_id] #4
CometProject [i_item_sk,i_item_id]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price]
- CometBroadcastExchange #5
- CometFilter [d_date,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_date] #5
+ CometFilter [d_date_sk,d_date]
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-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt
index d89167ec8..eb25d6d3e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt
@@ -9,12 +9,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
InputAdapter
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]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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
@@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [w_warehouse_sk] #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-spark4_0/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt
index 1ed007bc7..412f27ffb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt
@@ -28,7 +28,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -41,12 +41,12 @@ WholeStageCodegen (18)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date,count]
CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc]
CometProject [ss_item_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #6
@@ -54,14 +54,14 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #8
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
InputAdapter
WholeStageCodegen (6)
@@ -83,10 +83,10 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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
@@ -94,12 +94,12 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [c_customer_sk] #10
- CometBroadcastExchange #14
+ CometBroadcastExchange [d_date_sk] #14
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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
@@ -109,11 +109,11 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk]
CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #10
+ CometBroadcastExchange [c_customer_sk] #10
CometFilter [c_customer_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt
index 09a2bf7ce..d784ed47e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt
@@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
BroadcastHashJoin [cs_item_sk,item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date,count]
CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc]
CometProject [ss_item_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #6
@@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #8
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
InputAdapter
WholeStageCodegen (6)
@@ -84,10 +84,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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
@@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [c_customer_sk] #10
- CometBroadcastExchange #14
+ CometBroadcastExchange [d_date_sk] #14
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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
@@ -110,11 +110,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk]
CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #10
+ CometBroadcastExchange [c_customer_sk] #10
CometFilter [c_customer_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk]
InputAdapter
@@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
WholeStageCodegen (10)
@@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
BroadcastHashJoin [ws_item_sk,item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt
index 8ebd45fd1..30091beb8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt
@@ -39,7 +39,7 @@ WholeStageCodegen (11)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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
@@ -74,7 +74,7 @@ WholeStageCodegen (11)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -85,7 +85,7 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #5
@@ -93,26 +93,26 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,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]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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]
+ CometFilter [c_customer_sk,c_first_name,c_last_name,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]
+ CometFilter [ca_state,ca_zip,ca_country]
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-spark4_0/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt
index 8ebd45fd1..30091beb8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt
@@ -39,7 +39,7 @@ WholeStageCodegen (11)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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
@@ -74,7 +74,7 @@ WholeStageCodegen (11)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -85,7 +85,7 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #5
@@ -93,26 +93,26 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,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]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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]
+ CometFilter [c_customer_sk,c_first_name,c_last_name,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]
+ CometFilter [ca_state,ca_zip,ca_country]
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-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt
index 41efffcb4..6335e50ff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt
@@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name]
CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
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
@@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
- CometBroadcastExchange #9
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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-spark4_0/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt
index 67f335c66..b98e61704 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt
@@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id]
- CometBroadcastHashJoin [cs_promo_sk,p_promo_sk]
+ CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,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]
+ CometBroadcastHashJoin [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,cd_demo_sk]
+ CometFilter [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]
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
@@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [p_promo_sk] #6
CometProject [p_promo_sk]
- CometFilter [p_channel_email,p_channel_event,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_email,p_channel_event]
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-spark4_0/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt
index 74c8a1ca2..4d4c824bc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt
@@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
InputAdapter
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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
@@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [s_state,s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #5
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
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-spark4_0/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt
index a7a3f9537..2136886fe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt
@@ -17,7 +17,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -35,7 +35,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -53,7 +53,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -71,7 +71,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -89,7 +89,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -107,5 +107,5 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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-spark4_0/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt
index dbdfc8572..ee1f435d5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt
@@ -8,20 +8,20 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc]
CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
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
@@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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
@@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #8
+ CometBroadcastExchange [d_date_sk] #8
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk] #9
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #10
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
- CometBroadcastExchange #11
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt
index 3dc9ede25..36bb17a16 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt
@@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand]
ColumnarToRow
InputAdapter
CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id]
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-spark4_0/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt
index d5b8077ba..c4e59b49a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt
@@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
- CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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
@@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
@@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
- CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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
@@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [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]
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
@@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
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-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt
index c004db9b7..9040179da 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt
@@ -19,22 +19,22 @@ WholeStageCodegen (13)
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk,ca_county] #5
CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
InputAdapter
@@ -48,20 +48,20 @@ WholeStageCodegen (13)
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #9
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
ReusedExchange [ca_address_sk,ca_county] #5
InputAdapter
@@ -75,20 +75,20 @@ WholeStageCodegen (13)
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #13
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
ReusedExchange [ca_address_sk,ca_county] #5
InputAdapter
@@ -102,10 +102,10 @@ WholeStageCodegen (13)
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
@@ -121,10 +121,10 @@ WholeStageCodegen (13)
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
@@ -140,10 +140,10 @@ WholeStageCodegen (13)
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt
index 8cc26d8f4..b6ba1fbab 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt
@@ -11,8 +11,8 @@ WholeStageCodegen (5)
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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk]
+ CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk]
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
@@ -20,11 +20,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk] #3
CometProject [i_item_sk]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
InputAdapter
BroadcastExchange #4
@@ -38,13 +38,13 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_ext_discount_amt]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt
index 84143e015..bf0ef7ae4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt
@@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
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]
+ CometBroadcastExchange [i_item_sk,i_manufact_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_manufact_id] #7
CometProject [i_manufact_id]
- CometFilter [i_category]
+ CometFilter [i_category,i_manufact_id]
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]
@@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [cs_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt
index eefd38343..e1f7f9bd2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt
index efe0b0b4e..9deb51342 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt
@@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt
index 5bdf60081..58d617430 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt
@@ -17,12 +17,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
InputAdapter
CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id]
CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category]
CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #5
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt
index c3fcd79f0..efd86d5ab 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt
@@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
InputAdapter
CometHashAggregate [i_item_id,i_item_desc,i_current_price]
CometProject [i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometBroadcastExchange #2
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk]
- CometBroadcastHashJoin [i_item_sk,inv_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometFilter [i_current_price,i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
- CometBroadcastExchange #3
+ CometBroadcastExchange [inv_item_sk,inv_date_sk] #3
CometProject [inv_item_sk,inv_date_sk]
- CometFilter [inv_quantity_on_hand,inv_item_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [cs_item_sk]
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt
index cfac83844..9d667265c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt
@@ -15,10 +15,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ss_customer_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,14 +26,14 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_date] #4
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #6
@@ -46,10 +46,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [cs_bill_customer_sk,d_date]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_bill_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date]
+ CometFilter [cs_bill_customer_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #4
@@ -65,10 +65,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ws_bill_customer_sk,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt
index ff321c523..d16db5c8b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt
@@ -14,12 +14,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #3
@@ -27,17 +27,17 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk] #4
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_moy] #6
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #7
@@ -52,12 +52,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -65,11 +65,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
ReusedExchange [i_item_sk] #4
ReusedExchange [w_warehouse_sk,w_warehouse_name] #5
- CometBroadcastExchange #10
+ CometBroadcastExchange [d_date_sk,d_moy] #10
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt
index ff321c523..d16db5c8b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt
@@ -14,12 +14,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #3
@@ -27,17 +27,17 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk] #4
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_moy] #6
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #7
@@ -52,12 +52,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -65,11 +65,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
ReusedExchange [i_item_sk] #4
ReusedExchange [w_warehouse_sk,w_warehouse_name] #5
- CometBroadcastExchange #10
+ CometBroadcastExchange [d_date_sk,d_moy] #10
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt
index 0918da575..a412a54d9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt
@@ -18,23 +18,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -47,23 +47,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -77,13 +77,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 [cs_bill_customer_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12
+ CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -98,13 +98,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 [cs_bill_customer_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15
+ CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [d_date_sk,d_year] #9
@@ -120,13 +120,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 #18
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -141,13 +141,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 #21
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,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-spark4_0/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt
index 10e0735b4..11c0201a8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt
@@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [cs_warehouse_sk,cs_item_sk]
+ CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [w_warehouse_sk]
+ CometFilter [w_warehouse_sk,w_state]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state]
InputAdapter
BroadcastExchange #6
@@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_item_id]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price]
InputAdapter
ReusedExchange [d_date_sk,d_date] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt
index e31217066..457d441ce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt
@@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name]
ColumnarToRow
InputAdapter
CometProject [i_manufact,i_product_name]
- CometFilter [i_manufact_id,i_manufact]
+ CometFilter [i_manufact_id,i_manufact,i_product_name]
CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name]
InputAdapter
BroadcastExchange #2
@@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact]
+ CometHashAggregate [i_manufact,count]
CometProject [i_manufact]
- CometFilter [i_category,i_color,i_units,i_size,i_manufact]
+ CometFilter [i_category,i_manufact,i_size,i_color,i_units]
CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt
index e3967a972..d6e3733ce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt
@@ -8,16 +8,16 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category]
ColumnarToRow
InputAdapter
CometProject [d_year,ss_ext_sales_price,i_category_id,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3
CometProject [i_item_sk,i_category_id,i_category]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_category_id,i_category,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt
index 63945612a..f21d846c0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt
@@ -8,16 +8,16 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed
ColumnarToRow
InputAdapter
CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name]
CometProject [d_day_name,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_day_name]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name]
- CometBroadcastExchange #2
- CometFilter [ss_store_sk]
+ CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3
CometProject [s_store_sk,s_store_id,s_store_name]
- CometFilter [s_gmt_offset,s_store_sk]
+ CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt
index 35a3e9efa..12c8e0af9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt
@@ -35,7 +35,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_net_profit]
- CometFilter [ss_store_sk,ss_addr_sk]
+ CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
InputAdapter
@@ -45,7 +45,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_net_profit]
- CometFilter [ss_store_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
InputAdapter
WholeStageCodegen (10)
@@ -75,7 +75,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name]
InputAdapter
ReusedExchange [i_item_sk,i_product_name] #7
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt
index c376c4fcd..f60fdb18a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt
@@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [ws_sales_price,ca_city,ca_zip,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk]
CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip]
CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
- CometFilter [ws_bill_customer_sk,ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #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]
+ CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4
+ CometFilter [ca_address_sk,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
InputAdapter
BroadcastExchange #7
@@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [i_item_id]
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_item_id]
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-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt
index 718557c10..9ce3a98eb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt
@@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
ColumnarToRow
InputAdapter
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_city,s_store_sk]
+ CometFilter [s_store_sk,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk,ca_city] #6
CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
InputAdapter
@@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt
index 66eb5a4cb..1db8e7528 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt
@@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra
ColumnarToRow
InputAdapter
CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt
index 30f9822ab..23e083535 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt
@@ -7,14 +7,14 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
CometProject [ss_quantity]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk]
CometProject [ss_quantity,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit]
+ CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state]
CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -22,19 +22,19 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4
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 #5
+ CometBroadcastExchange [ca_address_sk,ca_state] #5
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt
index d1480ef10..bd3b21cdb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt
@@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
- CometBroadcastExchange #4
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
+ CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk]
- CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_return_amt,wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (10)
Project [item,return_ratio,return_rank,currency_rank]
@@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
+ CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk]
- CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_return_amount,cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
ReusedExchange [d_date_sk] #6
WholeStageCodegen (15)
@@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
- CometBroadcastExchange #12
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
+ CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk]
- CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
ReusedExchange [d_date_sk] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt
index 3dc3cca6c..ceb3e9213 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt
@@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sales_price,profit,return_amt,net_loss,s_store_id]
- CometBroadcastHashJoin [store_sk,s_store_sk]
+ CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id]
CometProject [store_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ss_store_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [sr_store_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
WholeStageCodegen (4)
HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum]
@@ -52,21 +52,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id]
- CometBroadcastHashJoin [page_sk,cp_catalog_page_sk]
+ CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id]
CometProject [page_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cs_catalog_page_sk]
+ CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cr_catalog_page_sk]
+ CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #7
- CometFilter [cp_catalog_page_sk]
+ CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
WholeStageCodegen (6)
HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum]
@@ -77,23 +77,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sales_price,profit,return_amt,net_loss,web_site_id]
- CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id]
CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ws_web_site_sk]
+ CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number]
+ CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_item_sk,ws_web_site_sk,ws_order_number]
- CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #10
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_site_id] #10
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt
index cd8a8ff7f..aa3f7f2cf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt
@@ -8,17 +8,17 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s
ColumnarToRow
InputAdapter
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk]
- CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4
+ CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometFilter [d_date_sk]
CometScan parquet spark_catalog.default.date_dim [d_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt
index 9ab7f18e7..f19312ca8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt
@@ -32,8 +32,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_sales_price,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_date] #6
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
InputAdapter
WholeStageCodegen (10)
@@ -69,8 +69,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_sales_price,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt
index d0b9da882..d7661b7be 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt
@@ -8,16 +8,16 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand]
ColumnarToRow
InputAdapter
CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt
index a04777412..b90cb42d0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt
@@ -17,16 +17,16 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [i_manufact_id,ss_sales_price,d_qoy]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk]
CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy]
CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [i_item_sk,i_manufact_id]
- CometFilter [i_category,i_class,i_brand,i_item_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_qoy] #5
CometProject [d_date_sk,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt
index 32e9a92a7..a8df48770 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt
@@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [c_customer_sk,c_current_addr_sk]
CometProject [c_customer_sk,c_current_addr_sk]
- CometBroadcastHashJoin [customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk]
CometProject [customer_sk]
- CometBroadcastHashJoin [sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk]
CometProject [sold_date_sk,customer_sk]
- CometBroadcastHashJoin [item_sk,i_item_sk]
- CometUnion
+ CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk]
+ CometUnion [sold_date_sk,customer_sk,item_sk]
CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk]
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk]
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #5
+ CometBroadcastExchange [i_item_sk] #5
CometProject [i_item_sk]
- CometFilter [i_category,i_class,i_item_sk]
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #7
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
InputAdapter
@@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [(d_month_seq + 1)] #3
ReusedSubquery [(d_month_seq + 3)] #4
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
@@ -85,7 +85,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [(d_month_seq + 1)]
CometProject [d_month_seq] [(d_month_seq + 1)]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
Subquery #4
WholeStageCodegen (2)
@@ -97,7 +97,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [(d_month_seq + 3)]
CometProject [d_month_seq] [(d_month_seq + 3)]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
InputAdapter
BroadcastExchange #12
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt
index a404b9b37..2461ab830 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt
@@ -8,16 +8,16 @@ TakeOrderedAndProject [ext_price,brand_id,brand]
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt
index a5cb59e57..c4ce35e15 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt
@@ -16,12 +16,12 @@ TakeOrderedAndProject [total_sales,i_item_id]
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #6
- CometBroadcastHashJoin [i_item_id,i_item_id]
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id]
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_item_id] #7
CometProject [i_item_id]
- CometFilter [i_color]
+ CometFilter [i_item_id,i_color]
CometScan parquet spark_catalog.default.item [i_item_id,i_color]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum]
@@ -56,12 +56,12 @@ TakeOrderedAndProject [total_sales,i_item_id]
ColumnarToRow
InputAdapter
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -76,12 +76,12 @@ TakeOrderedAndProject [total_sales,i_item_id]
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt
index ee91816c9..b64c2efdc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt
@@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_
ColumnarToRow
InputAdapter
CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [cs_item_sk,cs_call_center_sk]
+ CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3
+ CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #6
CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt
index 959d98ba9..0f5a783b9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt
@@ -13,10 +13,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -24,12 +24,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date] #3
CometProject [d_date]
- CometFilter [d_week_seq]
+ CometFilter [d_date,d_week_seq]
ReusedSubquery [d_week_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
Subquery #2
@@ -37,15 +37,15 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_date]
+ CometFilter [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk,i_item_id] #4
CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
ReusedExchange [d_date] #3
InputAdapter
@@ -60,10 +60,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
ColumnarToRow
InputAdapter
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk]
CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #4
@@ -80,10 +80,10 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk]
CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt
index 55cb85ddb..58bae46d1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt
@@ -14,18 +14,18 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name]
+ CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [d_date_sk,d_week_seq]
+ CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2
+ CometFilter [d_date_sk,d_week_seq,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk,s_store_id]
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
InputAdapter
BroadcastExchange #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt
index a7c65f50a..febbfd32f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt
@@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [ca_state,ss_item_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk]
CometProject [ca_state,ss_item_sk,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometProject [ca_state,c_customer_sk]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #2
- CometFilter [c_current_addr_sk,c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2
+ CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #3
- CometFilter [ss_customer_sk,ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
Subquery #2
@@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state]
InputAdapter
CometHashAggregate [d_month_seq]
CometProject [d_month_seq]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
@@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state]
BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
ColumnarToRow
InputAdapter
- CometFilter [i_current_price,i_category,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category]
InputAdapter
BroadcastExchange #8
@@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state]
HashAggregate [i_category,i_current_price] [sum,count,sum,count]
ColumnarToRow
InputAdapter
- CometFilter [i_category]
+ CometFilter [i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_current_price,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt
index cf226bb14..51025cf34 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt
@@ -16,12 +16,12 @@ TakeOrderedAndProject [i_item_id,total_sales]
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #6
- CometBroadcastHashJoin [i_item_id,i_item_id]
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id]
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_item_id] #7
CometProject [i_item_id]
- CometFilter [i_category]
+ CometFilter [i_item_id,i_category]
CometScan parquet spark_catalog.default.item [i_item_id,i_category]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum]
@@ -56,12 +56,12 @@ TakeOrderedAndProject [i_item_id,total_sales]
ColumnarToRow
InputAdapter
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -76,12 +76,12 @@ TakeOrderedAndProject [i_item_id,total_sales]
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt
index db13f94bd..8d39d9e4e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt
@@ -9,18 +9,18 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk]
CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -28,30 +28,30 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometProject [s_store_sk]
- CometFilter [s_gmt_offset,s_store_sk]
+ CometFilter [s_store_sk,s_gmt_offset]
CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset]
- CometBroadcastExchange #4
+ CometBroadcastExchange [p_promo_sk] #4
CometProject [p_promo_sk]
- CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #7
+ CometBroadcastExchange [ca_address_sk] #7
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #8
+ CometBroadcastExchange [i_item_sk] #8
CometProject [i_item_sk]
- CometFilter [i_category,i_item_sk]
+ CometFilter [i_item_sk,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category]
InputAdapter
BroadcastExchange #9
@@ -64,16 +64,16 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk]
CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [s_store_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt
index b25008e64..b9e35e282 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt
@@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days
ColumnarToRow
InputAdapter
CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression]
- CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk]
CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name]
- CometBroadcastHashJoin [ws_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name]
CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type]
- CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type]
CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk]
- CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #3
- CometFilter [sm_ship_mode_sk]
+ CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3
+ CometFilter [sm_ship_mode_sk,sm_type]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type]
- CometBroadcastExchange #4
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_name] #4
+ CometFilter [web_site_sk,web_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt
index a8eb9d7c6..e84d3c2a0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt
@@ -17,16 +17,16 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales]
ColumnarToRow
InputAdapter
CometProject [i_manager_id,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk]
CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy]
CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [i_item_sk,i_manager_id]
- CometFilter [i_category,i_class,i_brand,i_item_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_moy] #5
CometProject [d_date_sk,d_moy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt
index d972e0082..3b5e4f19e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt
@@ -54,19 +54,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #4
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -89,7 +89,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_ext_list_price]
- CometFilter [cs_item_sk,cs_order_number]
+ CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -100,7 +100,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #5
@@ -116,14 +116,14 @@ WholeStageCodegen (52)
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometFilter [d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
ReusedExchange [d_date_sk,d_year] #11
@@ -157,7 +157,7 @@ WholeStageCodegen (52)
WholeStageCodegen (19)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
InputAdapter
ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
@@ -176,7 +176,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_product_name]
- CometFilter [i_current_price,i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_color,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name]
InputAdapter
WholeStageCodegen (50)
@@ -227,19 +227,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #20
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #21
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (33)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt
index 1e34ce22f..84d23af42 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
BroadcastHashJoin [s_store_sk,ss_store_sk]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
InputAdapter
BroadcastExchange #1
@@ -22,8 +22,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand]
InputAdapter
BroadcastExchange #6
@@ -61,8 +61,8 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt
index 7d6c85af3..880c38bbf 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt
@@ -16,35 +16,35 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
ColumnarToRow
InputAdapter
CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk]
CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk]
CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy]
CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk]
- CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
+ CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4
+ CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastExchange #5
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [t_time_sk] #6
CometProject [t_time_sk]
- CometFilter [t_time,t_time_sk]
+ CometFilter [t_time_sk,t_time]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time]
- CometBroadcastExchange #7
+ CometBroadcastExchange [sm_ship_mode_sk] #7
CometProject [sm_ship_mode_sk]
- CometFilter [sm_carrier,sm_ship_mode_sk]
+ CometFilter [sm_ship_mode_sk,sm_carrier]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier]
WholeStageCodegen (4)
HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
@@ -55,14 +55,14 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
ColumnarToRow
InputAdapter
CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk]
CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk]
CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy]
CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk]
- CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
+ CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt
index 0386feff9..46d9cf4ef 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt
@@ -20,12 +20,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
InputAdapter
CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id]
CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy]
+ CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -33,15 +33,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt
index 1ecde47e8..78e1b1cce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt
@@ -12,14 +12,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
ColumnarToRow
InputAdapter
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_city,s_store_sk]
+ CometFilter [s_store_sk,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk,ca_city] #6
CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
InputAdapter
@@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt
index de2d5eeda..c9cc4959b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt
index 0429448f5..a27daca36 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt
@@ -8,14 +8,14 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id]
- CometBroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk]
CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_promo_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_item_sk,ss_promo_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [p_promo_sk] #6
CometProject [p_promo_sk]
- CometFilter [p_channel_email,p_channel_event,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_email,p_channel_event]
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-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt
index c962534aa..4a6dffc08 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt
@@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
BroadcastHashJoin [s_state,s_state]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_county,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state]
InputAdapter
BroadcastExchange #6
@@ -60,13 +60,13 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_net_profit,s_state]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk]
CometProject [ss_net_profit,ss_sold_date_sk,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #8
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #8
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt
index c732d903f..a4be53684 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt
@@ -11,17 +11,17 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute]
- CometBroadcastHashJoin [time_sk,t_time_sk]
+ CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute]
CometProject [i_brand_id,i_brand,ext_price,time_sk]
- CometBroadcastHashJoin [i_item_sk,sold_item_sk]
- CometBroadcastExchange #3
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
- CometUnion
+ CometUnion [ext_price,sold_item_sk,time_sk]
CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk,ws_sold_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -29,25 +29,25 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_item_sk,cs_sold_time_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_sold_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #6
+ CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6
CometProject [t_time_sk,t_hour,t_minute]
- CometFilter [t_meal_time,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute,t_meal_time]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt
index bea1fd4a1..6cba2d0e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt
@@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk]
- CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity]
- CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #4
- CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
+ CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #6
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
- CometBroadcastExchange #7
+ CometBroadcastExchange [cd_demo_sk] #7
CometProject [cd_demo_sk]
- CometFilter [cd_marital_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_marital_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status]
- CometBroadcastExchange #8
+ CometBroadcastExchange [hd_demo_sk] #8
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #10
- CometFilter [d_week_seq,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_week_seq] #10
+ CometFilter [d_date_sk,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
@@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt
index 060c3e153..c91c4cf47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt
index ea253444c..a7aa864ce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt
@@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #2
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #7
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #12
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #15
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,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-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt
index 34866bc37..27a4dcb67 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt
@@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
- CometBroadcastExchange #7
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #7
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
WholeStageCodegen (10)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
@@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
WholeStageCodegen (15)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
@@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (38)
@@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #17
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
- CometBroadcastExchange #18
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #18
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (23)
@@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt
index 6f8c28004..cfcf6a99a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt
@@ -7,32 +7,32 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_
HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum]
ColumnarToRow
InputAdapter
- CometUnion
+ CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_category] #2
+ CometFilter [i_item_sk,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category]
- CometBroadcastExchange #3
- CometFilter [d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_ship_customer_sk,ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category]
+ CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk]
ReusedExchange [i_item_sk,i_category] #2
ReusedExchange [d_date_sk,d_year,d_qoy] #3
CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_ship_addr_sk,cs_item_sk]
+ CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category]
+ CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedExchange [i_item_sk,i_category] #2
ReusedExchange [d_date_sk,d_year,d_qoy] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt
index 2989455d9..91786374e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt
@@ -19,10 +19,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
InputAdapter
@@ -50,10 +50,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_return_amt,sr_net_loss,s_store_sk]
- CometBroadcastHashJoin [sr_store_sk,s_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk]
CometProject [sr_store_sk,sr_return_amt,sr_net_loss]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- CometFilter [sr_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -84,7 +84,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_return_amount,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -99,14 +99,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #12
+ CometBroadcastExchange [wp_web_page_sk] #12
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
InputAdapter
@@ -120,10 +120,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk]
- CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk]
CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
- CometFilter [wr_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt
index 280687e30..13a8aecff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk,ss_customer_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt
index 5fcbcfed3..f2883e450 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt
@@ -10,12 +10,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1,
ColumnarToRow
InputAdapter
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1,
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk,s_city] #4
CometProject [s_store_sk,s_city]
- CometFilter [s_number_employees,s_store_sk]
+ CometFilter [s_store_sk,s_number_employees,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
InputAdapter
BroadcastExchange #6
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt
index adf555417..637f3b7ba 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt
@@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
ColumnarToRow
InputAdapter
CometProject [ss_net_profit,s_store_name,s_zip]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip]
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
- CometFilter [s_store_sk,s_zip]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4
+ CometFilter [s_store_sk,s_store_name,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip]
InputAdapter
BroadcastExchange #5
@@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_zip]
+ CometHashAggregate [ca_zip,count]
CometProject [ca_zip]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip]
- CometBroadcastExchange #9
+ CometBroadcastExchange [c_current_addr_sk] #9
CometProject [c_current_addr_sk]
- CometFilter [c_preferred_cust_flag,c_current_addr_sk]
+ CometFilter [c_current_addr_sk,c_preferred_cust_flag]
CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt
index 7e257bdc6..cdc8dc486 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt
@@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
InputAdapter
BroadcastExchange #7
@@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [i_item_sk]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price]
InputAdapter
BroadcastExchange #8
@@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [p_promo_sk]
- CometFilter [p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv]
WholeStageCodegen (20)
HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
@@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk]
+ CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cp_catalog_page_sk]
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
InputAdapter
ReusedExchange [i_item_sk] #7
@@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (21)
ColumnarToRow
InputAdapter
- CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss]
- CometFilter [wr_item_sk,wr_order_number]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (26)
ColumnarToRow
InputAdapter
- CometFilter [web_site_sk]
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
InputAdapter
ReusedExchange [i_item_sk] #7
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt
index 3daee59f8..991c00de9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt
@@ -15,10 +15,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
ColumnarToRow
InputAdapter
CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state]
- CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state]
CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
- CometFilter [cr_returning_addr_sk,cr_returning_customer_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk]
+ CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
@@ -52,10 +52,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
ColumnarToRow
InputAdapter
CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state]
- CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state]
CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
- CometFilter [cr_returning_addr_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk]
+ CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #3
@@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt
index a4d96e6f1..71a1c8e17 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt
@@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
InputAdapter
CometHashAggregate [i_item_id,i_item_desc,i_current_price]
CometProject [i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometBroadcastExchange #2
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk]
- CometBroadcastHashJoin [i_item_sk,inv_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometFilter [i_current_price,i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
- CometBroadcastExchange #3
+ CometBroadcastExchange [inv_item_sk,inv_date_sk] #3
CometProject [inv_item_sk,inv_date_sk]
- CometFilter [inv_quantity_on_hand,inv_item_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [ss_item_sk]
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt
index 7b7e907ee..8cd3f3f45 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt
@@ -12,10 +12,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
ColumnarToRow
InputAdapter
CometProject [sr_return_quantity,i_item_id]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk]
CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [sr_item_sk,i_item_sk]
- CometFilter [sr_item_sk]
+ CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date] #3
CometProject [d_date]
- CometBroadcastHashJoin [d_week_seq,d_week_seq]
+ CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_week_seq] #4
CometProject [d_week_seq]
- CometFilter [d_date]
+ CometFilter [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
ReusedExchange [d_date] #3
InputAdapter
@@ -54,10 +54,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
ColumnarToRow
InputAdapter
CometProject [cr_return_quantity,i_item_id]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk]
CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [cr_item_sk,i_item_sk]
- CometFilter [cr_item_sk]
+ CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #5
@@ -73,10 +73,10 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
ColumnarToRow
InputAdapter
CometProject [wr_return_quantity,i_item_id]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk]
CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [wr_item_sk,i_item_sk]
- CometFilter [wr_item_sk]
+ CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt
index fe7f7a207..646285a08 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt
@@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername]
Project [c_customer_id,c_last_name,c_first_name]
ColumnarToRow
InputAdapter
- CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk]
- CometBroadcastExchange #1
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk]
+ CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1
CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk]
- CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk]
CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk]
- CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk]
CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk]
CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk]
+ CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ca_address_sk] #2
CometProject [ca_address_sk]
- CometFilter [ca_city,ca_address_sk]
+ CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4
CometFilter [hd_demo_sk,hd_income_band_sk]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ib_income_band_sk] #5
CometProject [ib_income_band_sk]
- CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk]
+ CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound]
CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound]
CometProject [sr_cdemo_sk]
- CometFilter [sr_cdemo_sk]
+ CometFilter [sr_cdemo_sk,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt
index ad94ef82c..0d350bd59 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt
@@ -8,21 +8,21 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu
ColumnarToRow
InputAdapter
CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc]
- CometBroadcastHashJoin [wr_reason_sk,r_reason_sk]
+ CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc]
CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk]
CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit]
+ CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state]
CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status]
+ CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price]
+ CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number]
- CometBroadcastExchange #2
- CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit]
+ CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash]
+ CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2
+ CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash]
- CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk]
+ CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [wp_web_page_sk] #4
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5
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 #6
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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
+ CometBroadcastExchange [ca_address_sk,ca_state] #7
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #8
+ CometBroadcastExchange [d_date_sk] #8
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [r_reason_sk]
+ CometBroadcastExchange [r_reason_sk,r_reason_desc] #9
+ CometFilter [r_reason_sk,r_reason_desc]
CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt
index ea5dce5b6..e4d028f2f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt
@@ -17,10 +17,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
InputAdapter
CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id]
CometProject [ws_net_paid,i_category,i_class]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category]
CometProject [ws_item_sk,ws_net_paid]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #5
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt
index cfac83844..9d667265c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt
@@ -15,10 +15,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ss_customer_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,14 +26,14 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_date] #4
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #6
@@ -46,10 +46,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [cs_bill_customer_sk,d_date]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_bill_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date]
+ CometFilter [cs_bill_customer_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #4
@@ -65,10 +65,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ws_bill_customer_sk,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt
index b846d25d8..d3af10d19 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt
@@ -12,27 +12,27 @@ WholeStageCodegen (16)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_store_name,s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
InputAdapter
BroadcastExchange #5
@@ -43,20 +43,20 @@ WholeStageCodegen (16)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #7
+ CometBroadcastExchange [t_time_sk] #7
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -68,20 +68,20 @@ WholeStageCodegen (16)
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #10
+ CometBroadcastExchange [t_time_sk] #10
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -93,20 +93,20 @@ WholeStageCodegen (16)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #13
+ CometBroadcastExchange [t_time_sk] #13
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -118,20 +118,20 @@ WholeStageCodegen (16)
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #16
+ CometBroadcastExchange [t_time_sk] #16
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -143,20 +143,20 @@ WholeStageCodegen (16)
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #19
+ CometBroadcastExchange [t_time_sk] #19
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -168,20 +168,20 @@ WholeStageCodegen (16)
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #22
+ CometBroadcastExchange [t_time_sk] #22
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -193,19 +193,19 @@ WholeStageCodegen (16)
WholeStageCodegen (14)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #25
+ CometBroadcastExchange [t_time_sk] #25
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt
index 52e066fe7..2bfff90d3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt
@@ -17,15 +17,15 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla
ColumnarToRow
InputAdapter
CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy]
CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_category,i_class,i_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_moy] #5
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
+ CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt
index fdd3bd293..581739cc6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt
@@ -11,7 +11,7 @@ WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #1
ReusedSubquery [mergedValue] #1
@@ -26,7 +26,7 @@ WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #2
ReusedSubquery [mergedValue] #2
@@ -41,7 +41,7 @@ WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #3
ReusedSubquery [mergedValue] #3
@@ -56,7 +56,7 @@ WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #4
ReusedSubquery [mergedValue] #4
@@ -71,7 +71,7 @@ WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #5
ReusedSubquery [mergedValue] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt
index 50c8494fb..44159cc73 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt
@@ -7,27 +7,27 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
CometProject [ws_web_page_sk]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk]
CometProject [ws_sold_time_sk,ws_web_page_sk]
- CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk]
CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
- CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
+ CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_time_sk]
+ CometFilter [t_time_sk,t_hour]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour]
- CometBroadcastExchange #4
+ CometBroadcastExchange [wp_web_page_sk] #4
CometProject [wp_web_page_sk]
- CometFilter [wp_char_count,wp_web_page_sk]
+ CometFilter [wp_web_page_sk,wp_char_count]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count]
InputAdapter
BroadcastExchange #5
@@ -38,19 +38,19 @@ WholeStageCodegen (4)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
CometProject [ws_web_page_sk]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk]
CometProject [ws_sold_time_sk,ws_web_page_sk]
- CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk]
CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
- CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
+ CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #7
+ CometBroadcastExchange [t_time_sk] #7
CometProject [t_time_sk]
- CometFilter [t_hour,t_time_sk]
+ CometFilter [t_time_sk,t_hour]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour]
ReusedExchange [wp_web_page_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt
index 7e9b7b8ba..84f1190d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt
@@ -11,21 +11,21 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk]
- CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk]
- CometFilter [cc_call_center_sk]
+ CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
+ CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager]
- CometBroadcastExchange #3
- CometFilter [cr_call_center_sk,cr_returning_customer_sk]
+ CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3
+ CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -33,23 +33,23 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
- CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
- CometBroadcastExchange #7
+ CometBroadcastExchange [ca_address_sk] #7
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #8
- CometFilter [cd_marital_status,cd_education_status,cd_demo_sk]
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8
+ 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 #9
+ CometBroadcastExchange [hd_demo_sk] #9
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt
index bcc1ef1b5..aa24a049b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt
@@ -11,8 +11,8 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk,ws_ext_discount_amt]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk]
+ CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -20,11 +20,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk] #3
CometProject [i_item_sk]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
InputAdapter
BroadcastExchange #4
@@ -38,13 +38,13 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_ext_discount_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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-spark4_0/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt
index 3ec7ac7b6..a8eb231b7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt
@@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity]
- CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk]
+ CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
InputAdapter
BroadcastExchange #4
@@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk]
ColumnarToRow
InputAdapter
CometProject [r_reason_sk]
- CometFilter [r_reason_desc,r_reason_sk]
+ CometFilter [r_reason_sk,r_reason_desc]
CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt
index 34ddde768..d54c9e0c9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt
@@ -26,7 +26,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit]
- CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk]
+ CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (4)
@@ -54,7 +54,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -62,7 +62,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -70,5 +70,5 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [web_site_sk]
- CometFilter [web_company_name,web_site_sk]
+ CometFilter [web_site_sk,web_company_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt
index 5b699890c..178c25f66 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt
@@ -25,7 +25,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit]
- CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk]
+ CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (7)
@@ -40,7 +40,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ws_warehouse_sk,ws_order_number]
- CometFilter [ws_order_number,ws_warehouse_sk]
+ CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -60,7 +60,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [wr_order_number]
- CometFilter [wr_order_number]
+ CometFilter [wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (15)
@@ -82,7 +82,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -90,7 +90,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -98,5 +98,5 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [web_site_sk]
- CometFilter [web_company_name,web_site_sk]
+ CometFilter [web_site_sk,web_company_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt
index 614915226..e9d33a7f6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt
@@ -5,25 +5,25 @@ WholeStageCodegen (2)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_store_name,s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt
index 24e6dceef..41f3e579f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt
@@ -17,7 +17,7 @@ WholeStageCodegen (6)
InputAdapter
CometHashAggregate [ss_customer_sk,ss_item_sk]
CometProject [ss_item_sk,ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -25,11 +25,11 @@ WholeStageCodegen (6)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
WholeStageCodegen (4)
@@ -42,7 +42,7 @@ WholeStageCodegen (6)
InputAdapter
CometHashAggregate [cs_bill_customer_sk,cs_item_sk]
CometProject [cs_bill_customer_sk,cs_item_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt
index 84d277886..a92d99be4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt
@@ -20,10 +20,10 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -31,12 +31,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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-spark4_0/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt
index da4ef22e0..25ff3a2fe 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt
@@ -8,25 +8,25 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,
ColumnarToRow
InputAdapter
CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression]
- CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk]
CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name]
CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type]
- CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type]
CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk]
- CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #3
- CometFilter [sm_ship_mode_sk]
+ CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3
+ CometFilter [sm_ship_mode_sk,sm_type]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type]
- CometBroadcastExchange #4
- CometFilter [cc_call_center_sk]
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #4
+ CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt
index 000160518..238f9b702 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt
@@ -13,10 +13,10 @@ TakeOrderedAndProject [c_customer_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,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]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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
@@ -24,11 +24,11 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #4
@@ -45,10 +45,10 @@ TakeOrderedAndProject [c_customer_id]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt]
+ CometHashAggregate [sr_customer_sk,sr_store_sk,sum,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]
+ CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_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
@@ -58,12 +58,12 @@ TakeOrderedAndProject [c_customer_id]
ColumnarToRow
InputAdapter
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_customer_id]
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/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt
index 19243e359..efd4b187d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt
@@ -15,12 +15,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -60,12 +60,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_county,ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [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]
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/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt
index dd1a52206..127a5a291 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt
@@ -13,25 +13,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -42,25 +42,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -72,15 +72,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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
@@ -93,15 +93,15 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt
index 905a35c8b..4c2d3f3d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt
index 4de403664..98d995db0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt
@@ -5,18 +5,18 @@ WholeStageCodegen (2)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost]
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,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]
+ CometBroadcastHashJoin [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status,hd_demo_sk,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]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_demo_sk,cd_marital_status,cd_education_status]
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]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [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,s_store_sk]
+ CometFilter [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]
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
@@ -24,22 +24,22 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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
+ CometBroadcastExchange [hd_demo_sk,hd_dep_count] #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/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt
index e3dfa631b..2257d398f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt
@@ -19,20 +19,20 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -56,7 +56,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
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]
+ 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
BroadcastExchange #5
@@ -79,10 +79,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #7
@@ -90,26 +90,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #9
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10
+ 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 #11
+ CometBroadcastExchange [d_date_sk] #11
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #11
InputAdapter
@@ -118,10 +118,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -132,7 +132,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [i_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ 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] #4
@@ -154,7 +154,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [cs_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -179,7 +179,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
BroadcastHashJoin [ws_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk]
+ CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt
index 799f74a36..aa49638c7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt
@@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #2
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
@@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ 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
BroadcastExchange #4
@@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #3
BroadcastExchange #6
@@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ 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 #10
+ CometBroadcastExchange [d_date_sk] #10
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #10
InputAdapter
@@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #6
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt
index a03346372..b876fe4de 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt
@@ -6,14 +6,14 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_zip,cs_sales_price]
+ CometHashAggregate [ca_zip,sum,cs_sales_price]
CometProject [cs_sales_price,ca_zip]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,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]
+ CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_state,ca_zip]
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]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk,c_customer_sk,c_current_addr_sk]
+ CometFilter [cs_bill_customer_sk,cs_sales_price,cs_sold_date_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
@@ -21,15 +21,15 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #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]
+ CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4
+ CometFilter [ca_address_sk,ca_state,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
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/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt
index a55c182be..8935abb54 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt
@@ -26,7 +26,7 @@ WholeStageCodegen (12)
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]
+ CometFilter [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]
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)
@@ -54,7 +54,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -62,7 +62,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -70,5 +70,5 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [cc_call_center_sk]
- CometFilter [cc_county,cc_call_center_sk]
+ CometFilter [cc_call_center_sk,cc_county]
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/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt
index 77aba376e..ae6bab279 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt
@@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity]
+ CometHashAggregate [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,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_sk,i_item_id,i_item_desc]
CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_state]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
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
@@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_quarter_name,d_date_sk]
+ CometFilter [d_date_sk,d_quarter_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name]
ReusedExchange [d_date_sk] #7
- CometBroadcastExchange #8
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #8
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #9
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt
index 3d101857b..2519d58a2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt
@@ -6,21 +6,21 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
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]
+ CometHashAggregate [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,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]
+ CometBroadcastHashJoin [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,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_county,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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
@@ -28,26 +28,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3
CometProject [cd_demo_sk,cd_dep_count]
- CometFilter [cd_gender,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #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]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year]
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
+ CometBroadcastExchange [cd_demo_sk] #5
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #6
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6
+ CometFilter [ca_address_sk,ca_county,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #8
+ CometFilter [i_item_sk,i_item_id]
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/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt
index 18a69bcb4..163a31f47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt
@@ -6,33 +6,33 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price]
+ CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,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]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip,s_store_sk,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]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip]
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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact]
CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3
CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk,ca_zip] #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]
+ CometBroadcastExchange [s_store_sk,s_zip] #6
+ CometFilter [s_store_sk,s_zip]
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/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt
index c7999d981..5d0658192 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt
@@ -13,16 +13,16 @@ WholeStageCodegen (7)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_week_seq,d_day_name,sales_price]
+ CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price]
CometProject [sales_price,d_week_seq,d_day_name]
- CometBroadcastHashJoin [sold_date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name]
+ CometUnion [sold_date_sk,sales_price]
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]
+ CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3
+ CometFilter [d_date_sk,d_week_seq,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name]
InputAdapter
BroadcastExchange #4
@@ -30,7 +30,7 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_week_seq]
+ CometFilter [d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year]
InputAdapter
BroadcastExchange #5
@@ -46,5 +46,5 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_week_seq]
+ CometFilter [d_week_seq,d_year]
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/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt
index d805e3868..08088a386 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt
index 52bd7a85e..bd81a3a7b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt
@@ -7,29 +7,29 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,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]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id,d_date_sk,d_date]
CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk,i_item_id] #4
CometProject [i_item_sk,i_item_id]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price]
- CometBroadcastExchange #5
- CometFilter [d_date,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_date] #5
+ CometFilter [d_date_sk,d_date]
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/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt
index bda583c17..7d36dc400 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt
@@ -6,15 +6,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand]
+ CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,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]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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
@@ -22,15 +22,15 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [w_warehouse_sk] #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/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt
index f818fd25f..e065a0a87 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt
@@ -28,7 +28,7 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -41,12 +41,12 @@ WholeStageCodegen (18)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date,count]
CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc]
CometProject [ss_item_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #6
@@ -54,14 +54,14 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #8
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
InputAdapter
WholeStageCodegen (6)
@@ -81,12 +81,12 @@ WholeStageCodegen (18)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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
@@ -94,12 +94,12 @@ WholeStageCodegen (18)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [c_customer_sk] #10
- CometBroadcastExchange #14
+ CometBroadcastExchange [d_date_sk] #14
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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
@@ -107,13 +107,13 @@ WholeStageCodegen (18)
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk]
CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #10
+ CometBroadcastExchange [c_customer_sk] #10
CometFilter [c_customer_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt
index 54ee3dbde..2862f64cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt
@@ -21,7 +21,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
BroadcastHashJoin [cs_item_sk,item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -29,7 +29,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #4
@@ -42,12 +42,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,i_item_sk,d_date]
+ CometHashAggregate [_groupingexpression,i_item_sk,d_date,count]
CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc]
CometProject [ss_item_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #6
@@ -55,14 +55,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #8
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
InputAdapter
WholeStageCodegen (6)
@@ -82,12 +82,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_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
@@ -95,12 +95,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [c_customer_sk] #10
- CometBroadcastExchange #14
+ CometBroadcastExchange [d_date_sk] #14
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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
@@ -108,13 +108,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price]
+ CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price]
CometProject [ss_quantity,ss_sales_price,c_customer_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,c_customer_sk]
CometProject [ss_customer_sk,ss_quantity,ss_sales_price]
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #10
+ CometBroadcastExchange [c_customer_sk] #10
CometFilter [c_customer_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk]
InputAdapter
@@ -129,7 +129,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
WholeStageCodegen (10)
@@ -163,7 +163,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
BroadcastHashJoin [ws_item_sk,item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt
index 8ebd45fd1..30091beb8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt
@@ -39,7 +39,7 @@ WholeStageCodegen (11)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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
@@ -74,7 +74,7 @@ WholeStageCodegen (11)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -85,7 +85,7 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #5
@@ -93,26 +93,26 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,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]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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]
+ CometFilter [c_customer_sk,c_first_name,c_last_name,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]
+ CometFilter [ca_state,ca_zip,ca_country]
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/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt
index 8ebd45fd1..30091beb8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt
@@ -39,7 +39,7 @@ WholeStageCodegen (11)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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
@@ -74,7 +74,7 @@ WholeStageCodegen (11)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -85,7 +85,7 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #5
@@ -93,26 +93,26 @@ WholeStageCodegen (11)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,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]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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]
+ CometFilter [c_customer_sk,c_first_name,c_last_name,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]
+ CometFilter [ca_state,ca_zip,ca_country]
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/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt
index 74a5e8777..1f4faaac5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt
@@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
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]
+ CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_sk,s_store_id,s_store_name]
CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk]
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
@@ -40,24 +40,24 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
- CometBroadcastExchange #9
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt
index cba306f68..a39fbbb79 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt
@@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,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]
+ CometBroadcastHashJoin [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,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]
+ CometBroadcastHashJoin [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,cd_demo_sk]
+ CometFilter [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]
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
@@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [p_promo_sk] #6
CometProject [p_promo_sk]
- CometFilter [p_channel_email,p_channel_event,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_email,p_channel_event]
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/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt
index 7fcbe967a..9c4dd659a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt
@@ -6,17 +6,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price]
+ CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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
@@ -24,19 +24,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [s_state,s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #5
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
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/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt
index 47a4fe2c7..80c886708 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt
@@ -15,9 +15,9 @@ WholeStageCodegen (18)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -33,9 +33,9 @@ WholeStageCodegen (18)
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -51,9 +51,9 @@ WholeStageCodegen (18)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -69,9 +69,9 @@ WholeStageCodegen (18)
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -87,9 +87,9 @@ WholeStageCodegen (18)
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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
@@ -105,7 +105,7 @@ WholeStageCodegen (18)
WholeStageCodegen (15)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_list_price]
+ CometHashAggregate [ss_list_price,sum,count,count]
CometProject [ss_list_price]
- CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost]
+ CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
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/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt
index 2bcef6168..88b4ecd1d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt
@@ -6,22 +6,22 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity]
+ CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_sk,i_item_id,i_item_desc]
CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_sk,s_store_id,s_store_name]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_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
@@ -29,10 +29,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
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]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
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
@@ -40,10 +40,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [cs_bill_customer_sk,cs_item_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_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
@@ -51,23 +51,23 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #8
+ CometBroadcastExchange [d_date_sk] #8
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk] #9
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #10
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
- CometBroadcastExchange #11
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11
+ CometFilter [i_item_sk,i_item_id,i_item_desc]
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/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt
index cc9c4edf3..bccda58a4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price]
+ CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price]
CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id]
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/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt
index 50472502e..60cad82dd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt
@@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt]
+ CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt]
CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
- CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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
@@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
@@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt]
+ CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt]
CometProject [wr_returning_customer_sk,wr_return_amt,ca_state]
- CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state]
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]
+ CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_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
@@ -65,7 +65,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [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]
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
@@ -73,5 +73,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
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/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt
index a94a8a94d..a48c7ad26 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt
@@ -17,24 +17,24 @@ WholeStageCodegen (13)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk,ca_county] #5
CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
InputAdapter
@@ -46,22 +46,22 @@ WholeStageCodegen (13)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #9
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
ReusedExchange [ca_address_sk,ca_county] #5
InputAdapter
@@ -73,22 +73,22 @@ WholeStageCodegen (13)
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_addr_sk]
+ CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #13
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
ReusedExchange [ca_address_sk,ca_county] #5
InputAdapter
@@ -100,12 +100,12 @@ WholeStageCodegen (13)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
@@ -119,12 +119,12 @@ WholeStageCodegen (13)
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
@@ -138,12 +138,12 @@ WholeStageCodegen (13)
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price]
+ CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county]
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]
+ CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy]
+ CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt
index b8df1e929..c3bfdfd77 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt
@@ -11,8 +11,8 @@ WholeStageCodegen (5)
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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,i_item_sk]
+ CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk]
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
@@ -20,11 +20,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk] #3
CometProject [i_item_sk]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
InputAdapter
BroadcastExchange #4
@@ -36,15 +36,15 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_item_sk,cs_ext_discount_amt]
+ CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt]
CometProject [cs_item_sk,cs_ext_discount_amt]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_item_sk,cs_ext_discount_amt,cs_sold_date_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
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt
index af2b7cb5d..7a3c2c153 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt
@@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,ss_ext_sales_price]
+ CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
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]
+ CometBroadcastExchange [i_item_sk,i_manufact_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_manufact_id] #7
CometProject [i_manufact_id]
- CometFilter [i_category]
+ CometFilter [i_category,i_manufact_id]
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]
@@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,cs_ext_sales_price]
+ CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_manufact_id]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,ws_ext_sales_price]
+ CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt
index eefd38343..e1f7f9bd2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt
index efe0b0b4e..9deb51342 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt
@@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt
index 574a20fc0..4ad9b501b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt
@@ -14,15 +14,15 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price]
+ CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price]
CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id]
CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category]
CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,16 +30,16 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #5
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt
index c3fcd79f0..efd86d5ab 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt
@@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
InputAdapter
CometHashAggregate [i_item_id,i_item_desc,i_current_price]
CometProject [i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometBroadcastExchange #2
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,cs_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk]
- CometBroadcastHashJoin [i_item_sk,inv_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometFilter [i_current_price,i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
- CometBroadcastExchange #3
+ CometBroadcastExchange [inv_item_sk,inv_date_sk] #3
CometProject [inv_item_sk,inv_date_sk]
- CometFilter [inv_quantity_on_hand,inv_item_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [cs_item_sk]
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt
index cfac83844..9d667265c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt
@@ -15,10 +15,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ss_customer_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,14 +26,14 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_date] #4
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #6
@@ -46,10 +46,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [cs_bill_customer_sk,d_date]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_bill_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date]
+ CometFilter [cs_bill_customer_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #4
@@ -65,10 +65,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ws_bill_customer_sk,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt
index 7d28d5c10..0c709e4f7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #3
@@ -27,17 +27,17 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk] #4
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_moy] #6
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #7
@@ -50,14 +50,14 @@ WholeStageCodegen (5)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -65,11 +65,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
ReusedExchange [i_item_sk] #4
ReusedExchange [w_warehouse_sk,w_warehouse_name] #5
- CometBroadcastExchange #10
+ CometBroadcastExchange [d_date_sk,d_moy] #10
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt
index 7d28d5c10..0c709e4f7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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 #3
@@ -27,17 +27,17 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk] #4
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_moy] #6
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #7
@@ -50,14 +50,14 @@ WholeStageCodegen (5)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name,d_date_sk,d_moy]
CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name]
CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
- CometFilter [inv_item_sk,inv_warehouse_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -65,11 +65,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
ReusedExchange [i_item_sk] #4
ReusedExchange [w_warehouse_sk,w_warehouse_name] #5
- CometBroadcastExchange #10
+ CometBroadcastExchange [d_date_sk,d_moy] #10
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt
index 048da153b..1a19c58ff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt
@@ -16,25 +16,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (1)
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,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -45,25 +45,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (2)
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,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk,d_date_sk,d_year]
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_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -75,15 +75,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 [cs_bill_customer_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #12
+ CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -96,15 +96,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 [cs_bill_customer_sk]
+ CometBroadcastExchange [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] #15
+ CometFilter [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [d_date_sk,d_year] #9
@@ -118,15 +118,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (8)
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_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 #18
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #18
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -139,15 +139,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
WholeStageCodegen (10)
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_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk,d_date_sk,d_year]
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_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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 #21
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] #21
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,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/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt
index 10e0735b4..11c0201a8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt
@@ -21,14 +21,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [cs_warehouse_sk,cs_item_sk]
+ CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -39,14 +39,14 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [w_warehouse_sk]
+ CometFilter [w_warehouse_sk,w_state]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state]
InputAdapter
BroadcastExchange #6
@@ -54,7 +54,7 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_item_id]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price]
InputAdapter
ReusedExchange [d_date_sk,d_date] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt
index e31217066..457d441ce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt
@@ -10,7 +10,7 @@ TakeOrderedAndProject [i_product_name]
ColumnarToRow
InputAdapter
CometProject [i_manufact,i_product_name]
- CometFilter [i_manufact_id,i_manufact]
+ CometFilter [i_manufact_id,i_manufact,i_product_name]
CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name]
InputAdapter
BroadcastExchange #2
@@ -23,7 +23,7 @@ TakeOrderedAndProject [i_product_name]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact]
+ CometHashAggregate [i_manufact,count]
CometProject [i_manufact]
- CometFilter [i_category,i_color,i_units,i_size,i_manufact]
+ CometFilter [i_category,i_manufact,i_size,i_color,i_units]
CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt
index 3e69a3341..e3b850ef5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price]
+ CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price]
CometProject [d_year,ss_ext_sales_price,i_category_id,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_category_id,i_category]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3
CometProject [i_item_sk,i_category_id,i_category]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_category_id,i_category,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt
index d29a65bd4..5a060b85c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price]
+ CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price]
CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [d_day_name,ss_store_sk,ss_sales_price,s_store_sk,s_store_id,s_store_name]
CometProject [d_day_name,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_day_name]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name]
- CometBroadcastExchange #2
- CometFilter [ss_store_sk]
+ CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3
CometProject [s_store_sk,s_store_id,s_store_name]
- CometFilter [s_gmt_offset,s_store_sk]
+ CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt
index 701a90912..f713176f1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt
@@ -27,9 +27,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_store_sk,ss_net_profit]
+ CometHashAggregate [ss_store_sk,sum,count,ss_net_profit]
CometProject [ss_store_sk,ss_net_profit]
- CometFilter [ss_store_sk,ss_addr_sk]
+ CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
InputAdapter
@@ -37,9 +37,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,ss_net_profit]
+ CometHashAggregate [ss_item_sk,sum,count,ss_net_profit]
CometProject [ss_item_sk,ss_net_profit]
- CometFilter [ss_store_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -57,7 +57,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name]
InputAdapter
ReusedExchange [i_item_sk,i_product_name] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt
index c376c4fcd..f60fdb18a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt
@@ -11,14 +11,14 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [ws_sales_price,ca_city,ca_zip,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ca_city,ca_zip,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip,d_date_sk]
CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk,ca_address_sk,ca_city,ca_zip]
CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
- CometFilter [ws_bill_customer_sk,ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk,c_customer_sk,c_current_addr_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -26,20 +26,20 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #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]
+ CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4
+ CometFilter [ca_address_sk,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
InputAdapter
BroadcastExchange #7
@@ -47,5 +47,5 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)]
ColumnarToRow
InputAdapter
CometProject [i_item_id]
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_item_id]
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/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt
index 93e3eb05b..57defd961 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt
@@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_address_sk,ca_city]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_city,s_store_sk]
+ CometFilter [s_store_sk,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk,ca_city] #6
CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
InputAdapter
@@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt
index d2e615f39..b6e5c469b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price]
+ CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price]
CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt
index 6ebf6af07..5f628dbae 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt
@@ -5,16 +5,16 @@ WholeStageCodegen (2)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_quantity]
+ CometHashAggregate [sum,ss_quantity]
CometProject [ss_quantity]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk]
CometProject [ss_quantity,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit]
+ CometBroadcastHashJoin [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk,ca_address_sk,ca_state]
CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit]
+ CometBroadcastHashJoin [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -22,19 +22,19 @@ WholeStageCodegen (2)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4
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 #5
+ CometBroadcastExchange [ca_address_sk,ca_state] #5
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt
index 8d7b158d0..0e6b65b06 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
+ CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
- CometBroadcastExchange #4
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
+ CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk]
- CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_return_amt,wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (10)
Project [item,return_ratio,return_rank,currency_rank]
@@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
+ CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
+ CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk]
- CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_return_amount,cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
ReusedExchange [d_date_sk] #6
WholeStageCodegen (15)
@@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
+ CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
- CometBroadcastExchange #12
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
+ CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk]
- CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
ReusedExchange [d_date_sk] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt
index c4ea8fe24..537cba446 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt
@@ -15,14 +15,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,s_store_id]
- CometBroadcastHashJoin [store_sk,s_store_sk]
+ CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id]
CometProject [store_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ss_store_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,18 +30,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [sr_store_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
WholeStageCodegen (4)
HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum]
@@ -50,23 +50,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id]
- CometBroadcastHashJoin [page_sk,cp_catalog_page_sk]
+ CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id]
CometProject [page_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cs_catalog_page_sk]
+ CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cr_catalog_page_sk]
+ CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #7
- CometFilter [cp_catalog_page_sk]
+ CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
WholeStageCodegen (6)
HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum]
@@ -75,25 +75,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,web_site_id]
- CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id]
CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ws_web_site_sk]
+ CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number]
+ CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_item_sk,ws_web_site_sk,ws_order_number]
- CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #10
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_site_id] #10
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt
index 473b9cdd4..58adabcec 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt
@@ -6,19 +6,19 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk]
+ CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,d_date_sk]
CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk,s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk]
- CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk]
+ CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] #2
+ CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,15 +26,15 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4
+ CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometFilter [d_date_sk]
CometScan parquet spark_catalog.default.date_dim [d_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt
index ce6005da6..988297f02 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt
@@ -30,10 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,d_date,ws_sales_price]
+ CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price]
CometProject [ws_item_sk,ws_sales_price,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -41,11 +41,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk,d_date] #6
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
InputAdapter
WholeStageCodegen (10)
@@ -67,10 +67,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,d_date,ss_sales_price]
+ CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price]
CometProject [ss_item_sk,ss_sales_price,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt
index 557dd3b4a..a0932c7cc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price]
+ CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price]
CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [d_year,ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [d_year,ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk,d_year]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt
index 83e53bb66..b2199fb34 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt
@@ -15,18 +15,18 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price]
+ CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price]
CometProject [i_manufact_id,ss_sales_price,d_qoy]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk]
CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy]
CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [i_item_sk,i_manufact_id]
- CometFilter [i_category,i_class,i_brand,i_item_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -34,12 +34,12 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_qoy] #5
CometProject [d_date_sk,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt
index c6886735f..2193d875c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt
@@ -26,14 +26,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [c_customer_sk,c_current_addr_sk]
CometProject [c_customer_sk,c_current_addr_sk]
- CometBroadcastHashJoin [customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk]
CometProject [customer_sk]
- CometBroadcastHashJoin [sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk]
CometProject [sold_date_sk,customer_sk]
- CometBroadcastHashJoin [item_sk,i_item_sk]
- CometUnion
+ CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk]
+ CometUnion [sold_date_sk,customer_sk,item_sk]
CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk]
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -41,21 +41,21 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk]
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #5
+ CometBroadcastExchange [i_item_sk] #5
CometProject [i_item_sk]
- CometFilter [i_category,i_class,i_item_sk]
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #7
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
InputAdapter
@@ -63,7 +63,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [ss_customer_sk]
+ CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #9
@@ -71,7 +71,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
Subquery #3
WholeStageCodegen (2)
HashAggregate [(d_month_seq + 1)]
@@ -82,7 +82,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [(d_month_seq + 1)]
CometProject [d_month_seq] [(d_month_seq + 1)]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
Subquery #4
WholeStageCodegen (2)
@@ -94,7 +94,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base]
InputAdapter
CometHashAggregate [(d_month_seq + 3)]
CometProject [d_month_seq] [(d_month_seq + 3)]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt
index 2750a6ba2..fc38884c1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt
@@ -6,18 +6,18 @@ TakeOrderedAndProject [ext_price,brand_id,brand]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price]
+ CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_brand_id,i_brand]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk]
+ CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #2
- CometFilter [ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt
index 7fdead831..0b48046fd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt
@@ -14,14 +14,14 @@ TakeOrderedAndProject [total_sales,i_item_id]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [total_sales,i_item_id]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #6
- CometBroadcastHashJoin [i_item_id,i_item_id]
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id]
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_item_id] #7
CometProject [i_item_id]
- CometFilter [i_color]
+ CometFilter [i_item_id,i_color]
CometScan parquet spark_catalog.default.item [i_item_id,i_color]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum]
@@ -54,14 +54,14 @@ TakeOrderedAndProject [total_sales,i_item_id]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -74,14 +74,14 @@ TakeOrderedAndProject [total_sales,i_item_id]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt
index c630cad48..fb2b09b9d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price]
+ CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price]
CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [cs_item_sk,cs_call_center_sk]
+ CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3
+ CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #6
CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt
index 2ed2bde44..fc7c21e94 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt
@@ -11,12 +11,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -24,27 +24,27 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date] #3
CometProject [d_date]
- CometFilter [d_week_seq]
+ CometFilter [d_date,d_week_seq]
Subquery #2
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_date]
+ CometFilter [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [i_item_sk,i_item_id] #4
CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
ReusedExchange [d_date] #3
InputAdapter
@@ -57,12 +57,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk]
CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #4
@@ -77,12 +77,12 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk]
CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt
index e00d52dbb..6c1e6cd92 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt
@@ -12,20 +12,20 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price]
+ CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price]
CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_week_seq,d_day_name]
+ CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [d_date_sk,d_week_seq]
+ CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2
+ CometFilter [d_date_sk,d_week_seq,d_day_name]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name]
InputAdapter
BroadcastExchange #3
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk,s_store_id]
+ CometFilter [s_store_sk,s_store_id,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name]
InputAdapter
BroadcastExchange #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt
index 89a080d85..824670101 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt
@@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [ca_state,ss_item_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk]
CometProject [ca_state,ss_item_sk,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometProject [ca_state,c_customer_sk]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #2
- CometFilter [c_current_addr_sk,c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2
+ CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #3
- CometFilter [ss_customer_sk,ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
Subquery #2
WholeStageCodegen (2)
HashAggregate [d_month_seq]
@@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,state]
InputAdapter
CometHashAggregate [d_month_seq]
CometProject [d_month_seq]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
@@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,state]
BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
ColumnarToRow
InputAdapter
- CometFilter [i_current_price,i_category,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category]
InputAdapter
BroadcastExchange #8
@@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,state]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_current_price]
- CometFilter [i_category]
+ CometHashAggregate [i_category,sum,count,i_current_price]
+ CometFilter [i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_current_price,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt
index b76e7c9b3..7cfcb75da 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt
@@ -14,14 +14,14 @@ TakeOrderedAndProject [i_item_id,total_sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_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
@@ -29,23 +29,23 @@ TakeOrderedAndProject [i_item_id,total_sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ca_address_sk] #5
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #6
- CometBroadcastHashJoin [i_item_id,i_item_id]
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id]
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #7
+ CometBroadcastExchange [i_item_id] #7
CometProject [i_item_id]
- CometFilter [i_category]
+ CometFilter [i_item_id,i_category]
CometScan parquet spark_catalog.default.item [i_item_id,i_category]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum]
@@ -54,14 +54,14 @@ TakeOrderedAndProject [i_item_id,total_sales]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,cs_ext_sales_price]
CometProject [cs_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_ext_sales_price]
- CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,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]
+ CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -74,14 +74,14 @@ TakeOrderedAndProject [i_item_id,total_sales]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,sum,ws_ext_sales_price]
CometProject [ws_ext_sales_price,i_item_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id]
CometProject [ws_item_sk,ws_ext_sales_price]
- CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt
index 4ca1dd667..1b2af33b0 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt
@@ -7,20 +7,20 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_sales_price]
+ CometHashAggregate [sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk]
CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,p_promo_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -28,30 +28,30 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [s_store_sk] #3
CometProject [s_store_sk]
- CometFilter [s_gmt_offset,s_store_sk]
+ CometFilter [s_store_sk,s_gmt_offset]
CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset]
- CometBroadcastExchange #4
+ CometBroadcastExchange [p_promo_sk] #4
CometProject [p_promo_sk]
- CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6
CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #7
+ CometBroadcastExchange [ca_address_sk] #7
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #8
+ CometBroadcastExchange [i_item_sk] #8
CometProject [i_item_sk]
- CometFilter [i_category,i_item_sk]
+ CometFilter [i_item_sk,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category]
InputAdapter
BroadcastExchange #9
@@ -62,18 +62,18 @@ WholeStageCodegen (4)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_sales_price]
+ CometHashAggregate [sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk]
CometProject [ss_item_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,c_current_addr_sk,ca_address_sk]
CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,c_customer_sk,c_current_addr_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,s_store_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [s_store_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt
index c6b7e1834..0b4fc61a1 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt
@@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk]
+ CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk]
CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression]
- CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name,d_date_sk]
CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name]
- CometBroadcastHashJoin [ws_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name]
CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type]
- CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type]
CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk]
- CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk]
+ CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #3
- CometFilter [sm_ship_mode_sk]
+ CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3
+ CometFilter [sm_ship_mode_sk,sm_type]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type]
- CometBroadcastExchange #4
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_name] #4
+ CometFilter [web_site_sk,web_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt
index 35e09ec7d..b2033c7a9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt
@@ -15,18 +15,18 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_manager_id,d_moy,ss_sales_price]
+ CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price]
CometProject [i_manager_id,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk]
CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy]
CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometProject [i_item_sk,i_manager_id]
- CometFilter [i_category,i_class,i_brand,i_item_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -34,12 +34,12 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales]
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_moy] #5
CometProject [d_date_sk,d_moy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt
index d972e0082..3b5e4f19e 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt
@@ -54,19 +54,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #4
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -89,7 +89,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_ext_list_price]
- CometFilter [cs_item_sk,cs_order_number]
+ CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -100,7 +100,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #5
@@ -116,14 +116,14 @@ WholeStageCodegen (52)
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometFilter [d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
ReusedExchange [d_date_sk,d_year] #11
@@ -157,7 +157,7 @@ WholeStageCodegen (52)
WholeStageCodegen (19)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
InputAdapter
ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
@@ -176,7 +176,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_product_name]
- CometFilter [i_current_price,i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_color,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name]
InputAdapter
WholeStageCodegen (50)
@@ -227,19 +227,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #20
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #21
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (33)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt
index 8de564ed1..6cb247973 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
BroadcastHashJoin [s_store_sk,ss_store_sk]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
InputAdapter
BroadcastExchange #1
@@ -20,10 +20,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price]
+ CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price]
CometProject [ss_item_sk,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -31,18 +31,18 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #5
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand]
InputAdapter
BroadcastExchange #6
@@ -59,10 +59,10 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price]
+ CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price]
CometProject [ss_item_sk,ss_store_sk,ss_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt
index d746739b3..72133811b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt
@@ -14,37 +14,37 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid]
CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk]
CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk]
CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy]
CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk]
- CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
+ CometFilter [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4
+ CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastExchange #5
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [t_time_sk] #6
CometProject [t_time_sk]
- CometFilter [t_time,t_time_sk]
+ CometFilter [t_time_sk,t_time]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time]
- CometBroadcastExchange #7
+ CometBroadcastExchange [sm_ship_mode_sk] #7
CometProject [sm_ship_mode_sk]
- CometFilter [sm_carrier,sm_ship_mode_sk]
+ CometFilter [sm_ship_mode_sk,sm_carrier]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier]
WholeStageCodegen (4)
HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty]
@@ -53,16 +53,16 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax]
+ CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax]
CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,sm_ship_mode_sk]
CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,t_time_sk]
CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_date_sk,d_year,d_moy]
CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
- CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk]
- CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
+ CometFilter [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
index b529cb5ff..c37569802 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
@@ -14,15 +14,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity]
+ CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty,ss_sales_price,ss_quantity]
CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id]
CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy]
+ CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,15 +30,15 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt
index 43f44c9f7..c5f56a69c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt
@@ -10,16 +10,16 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city]
- CometBroadcastHashJoin [ss_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_address_sk,ca_city]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,s_store_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -27,21 +27,21 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_city,s_store_sk]
+ CometFilter [s_store_sk,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk,ca_city] #6
CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
InputAdapter
@@ -49,7 +49,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt
index de2d5eeda..c9cc4959b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -27,11 +27,11 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -39,7 +39,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -49,7 +49,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt
index c583ba8e8..f327d5b06 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt
@@ -6,16 +6,16 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price]
CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id]
- CometBroadcastHashJoin [ss_promo_sk,p_promo_sk]
+ CometBroadcastHashJoin [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,p_promo_sk]
CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk]
CometProject [ss_item_sk,ss_promo_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_item_sk,ss_promo_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,20 +23,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [p_promo_sk] #6
CometProject [p_promo_sk]
- CometFilter [p_channel_email,p_channel_event,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_email,p_channel_event]
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/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt
index f4a03a2d2..f95e8d040 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt
@@ -19,8 +19,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -28,11 +28,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
BroadcastHashJoin [s_state,s_state]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_county,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state]
InputAdapter
BroadcastExchange #6
@@ -57,15 +57,15 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_state,ss_net_profit]
+ CometHashAggregate [s_state,sum,ss_net_profit]
CometProject [ss_net_profit,s_state]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk]
CometProject [ss_net_profit,ss_sold_date_sk,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #8
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #8
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt
index fd6777886..c604a8fa7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt
@@ -9,19 +9,19 @@ WholeStageCodegen (3)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price]
+ CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price]
CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute]
- CometBroadcastHashJoin [time_sk,t_time_sk]
+ CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute]
CometProject [i_brand_id,i_brand,ext_price,time_sk]
- CometBroadcastHashJoin [i_item_sk,sold_item_sk]
- CometBroadcastExchange #3
+ CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3
CometProject [i_item_sk,i_brand_id,i_brand]
- CometFilter [i_manager_id,i_item_sk]
+ CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id]
- CometUnion
+ CometUnion [ext_price,sold_item_sk,time_sk]
CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk,ws_sold_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -29,25 +29,25 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_moy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_item_sk,cs_sold_time_sk]
+ CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk]
+ CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_sold_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #6
+ CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6
CometProject [t_time_sk,t_hour,t_minute]
- CometFilter [t_meal_time,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute,t_meal_time]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt
index bea1fd4a1..6cba2d0e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt
@@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk]
- CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity]
- CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #4
- CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
+ CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #6
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
- CometBroadcastExchange #7
+ CometBroadcastExchange [cd_demo_sk] #7
CometProject [cd_demo_sk]
- CometFilter [cd_marital_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_marital_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status]
- CometBroadcastExchange #8
+ CometBroadcastExchange [hd_demo_sk] #8
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #10
- CometFilter [d_week_seq,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_week_seq] #10
+ CometFilter [d_date_sk,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
@@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt
index 060c3e153..c91c4cf47 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt
index 2e6286d93..7c2a42ca3 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt
@@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #2
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #7
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #12
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #15
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,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/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt
index 34866bc37..27a4dcb67 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt
@@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
- CometBroadcastExchange #7
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #7
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
WholeStageCodegen (10)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
@@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
WholeStageCodegen (15)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
@@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (38)
@@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #17
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
- CometBroadcastExchange #18
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #18
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (23)
@@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt
index 473eef9e4..e463296d4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt
@@ -6,33 +6,33 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometUnion
+ CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price]
+ CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_category] #2
+ CometFilter [i_item_sk,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_category]
- CometBroadcastExchange #3
- CometFilter [d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_ship_customer_sk,ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category]
+ CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk]
ReusedExchange [i_item_sk,i_category] #2
ReusedExchange [d_date_sk,d_year,d_qoy] #3
CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy]
CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_ship_addr_sk,cs_item_sk]
+ CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category]
+ CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk]
ReusedExchange [i_item_sk,i_category] #2
ReusedExchange [d_date_sk,d_year,d_qoy] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt
index 590c59fdc..f9088f784 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt
@@ -17,12 +17,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit]
+ CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit]
CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,13 +30,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
InputAdapter
@@ -48,12 +48,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss]
+ CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss]
CometProject [sr_return_amt,sr_net_loss,s_store_sk]
- CometBroadcastHashJoin [sr_store_sk,s_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk]
CometProject [sr_store_sk,sr_return_amt,sr_net_loss]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- CometFilter [sr_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -70,9 +70,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
+ CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit]
CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_return_amount,cr_net_loss]
+ CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss]
CometProject [cr_return_amount,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
@@ -97,16 +97,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit]
+ CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit]
CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #12
+ CometBroadcastExchange [wp_web_page_sk] #12
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
InputAdapter
@@ -118,12 +118,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss]
+ CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss]
CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk]
- CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk]
CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
- CometFilter [wr_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt
index 280687e30..13a8aecff 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk,ss_customer_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -45,7 +45,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -83,7 +83,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -109,7 +109,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt
index b68a9474b..57598ec35 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt
@@ -8,14 +8,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1,
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit]
CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_store_sk,s_city]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,24 +23,24 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1,
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_dow,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dow]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk,s_city] #4
CometProject [s_store_sk,s_city]
- CometFilter [s_number_employees,s_store_sk]
+ CometFilter [s_store_sk,s_number_employees,s_city]
CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city]
- CometBroadcastExchange #5
+ CometBroadcastExchange [hd_demo_sk] #5
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
InputAdapter
BroadcastExchange #6
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk]
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt
index adf555417..637f3b7ba 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt
@@ -10,10 +10,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
ColumnarToRow
InputAdapter
CometProject [ss_net_profit,s_store_name,s_zip]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip]
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -21,14 +21,14 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_qoy,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
- CometFilter [s_store_sk,s_zip]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4
+ CometFilter [s_store_sk,s_store_name,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip]
InputAdapter
BroadcastExchange #5
@@ -55,12 +55,12 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_zip]
+ CometHashAggregate [ca_zip,count]
CometProject [ca_zip]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip]
- CometBroadcastExchange #9
+ CometBroadcastExchange [c_current_addr_sk] #9
CometProject [c_current_addr_sk]
- CometFilter [c_preferred_cust_flag,c_current_addr_sk]
+ CometFilter [c_current_addr_sk,c_preferred_cust_flag]
CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt
index 7e257bdc6..cdc8dc486 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt
@@ -32,7 +32,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -51,7 +51,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
InputAdapter
BroadcastExchange #7
@@ -68,7 +68,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [i_item_sk]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price]
InputAdapter
BroadcastExchange #8
@@ -76,7 +76,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [p_promo_sk]
- CometFilter [p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv]
WholeStageCodegen (20)
HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
@@ -102,7 +102,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk]
+ CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -114,7 +114,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -123,7 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cp_catalog_page_sk]
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
InputAdapter
ReusedExchange [i_item_sk] #7
@@ -153,7 +153,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (21)
ColumnarToRow
InputAdapter
- CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -165,7 +165,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss]
- CometFilter [wr_item_sk,wr_order_number]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #4
@@ -174,7 +174,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (26)
ColumnarToRow
InputAdapter
- CometFilter [web_site_sk]
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
InputAdapter
ReusedExchange [i_item_sk] #7
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt
index f80554fb5..e576a079b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt
@@ -13,12 +13,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax]
+ CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax]
CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state]
- CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state]
CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
- CometFilter [cr_returning_addr_sk,cr_returning_customer_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk]
+ CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -26,13 +26,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
+ CometBroadcastExchange [ca_address_sk,ca_state] #4
CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
@@ -50,12 +50,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax]
+ CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax]
CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state]
- CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state]
CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
- CometFilter [cr_returning_addr_sk]
+ CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk]
+ CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #3
@@ -65,12 +65,12 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt
index a4d96e6f1..71a1c8e17 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt
@@ -8,18 +8,18 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
InputAdapter
CometHashAggregate [i_item_id,i_item_desc,i_current_price]
CometProject [i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometBroadcastExchange #2
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,ss_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price] #2
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk,d_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk]
- CometBroadcastHashJoin [i_item_sk,inv_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk]
CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price]
- CometFilter [i_current_price,i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id]
- CometBroadcastExchange #3
+ CometBroadcastExchange [inv_item_sk,inv_date_sk] #3
CometProject [inv_item_sk,inv_date_sk]
- CometFilter [inv_quantity_on_hand,inv_item_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [ss_item_sk]
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt
index ef7d35e21..390c20b7f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt
@@ -10,12 +10,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,sr_return_quantity]
+ CometHashAggregate [i_item_id,sum,sr_return_quantity]
CometProject [sr_return_quantity,i_item_id]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [sr_return_quantity,sr_returned_date_sk,i_item_id,d_date_sk]
CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [sr_item_sk,i_item_sk]
- CometFilter [sr_item_sk]
+ CometBroadcastHashJoin [sr_item_sk,sr_return_quantity,sr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [sr_item_sk,sr_return_quantity,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -23,24 +23,24 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date] #3
CometProject [d_date]
- CometBroadcastHashJoin [d_week_seq,d_week_seq]
+ CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_week_seq] #4
CometProject [d_week_seq]
- CometFilter [d_date]
+ CometFilter [d_date,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [i_item_sk,i_item_id] #5
CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometBroadcastHashJoin [d_date,d_date]
- CometFilter [d_date_sk]
+ CometBroadcastHashJoin [d_date_sk,d_date,d_date]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
ReusedExchange [d_date] #3
InputAdapter
@@ -52,12 +52,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,cr_return_quantity]
+ CometHashAggregate [i_item_id,sum,cr_return_quantity]
CometProject [cr_return_quantity,i_item_id]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_quantity,cr_returned_date_sk,i_item_id,d_date_sk]
CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [cr_item_sk,i_item_sk]
- CometFilter [cr_item_sk]
+ CometBroadcastHashJoin [cr_item_sk,cr_return_quantity,cr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [cr_item_sk,cr_return_quantity,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #5
@@ -71,12 +71,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,wr_return_quantity]
+ CometHashAggregate [i_item_id,sum,wr_return_quantity]
CometProject [wr_return_quantity,i_item_id]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [wr_return_quantity,wr_returned_date_sk,i_item_id,d_date_sk]
CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id]
- CometBroadcastHashJoin [wr_item_sk,i_item_sk]
- CometFilter [wr_item_sk]
+ CometBroadcastHashJoin [wr_item_sk,wr_return_quantity,wr_returned_date_sk,i_item_sk,i_item_id]
+ CometFilter [wr_item_sk,wr_return_quantity,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_item_id] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt
index fe7f7a207..646285a08 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt
@@ -3,32 +3,32 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername]
Project [c_customer_id,c_last_name,c_first_name]
ColumnarToRow
InputAdapter
- CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk]
- CometBroadcastExchange #1
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk]
+ CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1
CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk]
- CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk]
CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk]
- CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk]
CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk]
CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk]
+ CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk]
+ CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ca_address_sk] #2
CometProject [ca_address_sk]
- CometFilter [ca_city,ca_address_sk]
+ CometFilter [ca_address_sk,ca_city]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4
CometFilter [hd_demo_sk,hd_income_band_sk]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [ib_income_band_sk] #5
CometProject [ib_income_band_sk]
- CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk]
+ CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound]
CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound]
CometProject [sr_cdemo_sk]
- CometFilter [sr_cdemo_sk]
+ CometFilter [sr_cdemo_sk,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt
index ecae29c78..2313467ae 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt
@@ -6,23 +6,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee]
+ CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee]
CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc]
- CometBroadcastHashJoin [wr_reason_sk,r_reason_sk]
+ CometBroadcastHashJoin [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash,r_reason_sk,r_reason_desc]
CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash,d_date_sk]
CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit]
+ CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash,ca_address_sk,ca_state]
CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status]
+ CometBroadcastHashJoin [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price]
+ CometBroadcastHashJoin [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash]
- CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number]
- CometBroadcastExchange #2
- CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit]
+ CometBroadcastHashJoin [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash]
+ CometBroadcastExchange [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] #2
+ CometFilter [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,28 +30,28 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash]
- CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk]
+ CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk]
- CometBroadcastExchange #4
+ CometBroadcastExchange [wp_web_page_sk] #4
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
- CometBroadcastExchange #5
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5
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 #6
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #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
+ CometBroadcastExchange [ca_address_sk,ca_state] #7
CometProject [ca_address_sk,ca_state]
- CometFilter [ca_country,ca_address_sk,ca_state]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
- CometBroadcastExchange #8
+ CometBroadcastExchange [d_date_sk] #8
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [r_reason_sk]
+ CometBroadcastExchange [r_reason_sk,r_reason_desc] #9
+ CometFilter [r_reason_sk,r_reason_desc]
CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt
index 4218938c1..a6d1c25fc 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt
@@ -14,13 +14,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid]
+ CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid]
CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id]
CometProject [ws_net_paid,i_category,i_class]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category]
CometProject [ws_item_sk,ws_net_paid]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -28,12 +28,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #5
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt
index cfac83844..9d667265c 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt
@@ -15,10 +15,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ss_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ss_customer_sk,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_customer_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -26,14 +26,14 @@ WholeStageCodegen (7)
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_date] #4
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #5
- CometFilter [c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5
+ CometFilter [c_customer_sk,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name]
InputAdapter
BroadcastExchange #6
@@ -46,10 +46,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [cs_bill_customer_sk,d_date]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
- CometFilter [cs_bill_customer_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date]
+ CometFilter [cs_bill_customer_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #4
@@ -65,10 +65,10 @@ WholeStageCodegen (7)
InputAdapter
CometHashAggregate [c_last_name,c_first_name,d_date]
CometProject [c_last_name,c_first_name,d_date]
- CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name]
CometProject [ws_bill_customer_sk,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_bill_customer_sk,ws_sold_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,d_date] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt
index b846d25d8..d3af10d19 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt
@@ -12,27 +12,27 @@ WholeStageCodegen (16)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_store_name,s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
InputAdapter
BroadcastExchange #5
@@ -43,20 +43,20 @@ WholeStageCodegen (16)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #7
+ CometBroadcastExchange [t_time_sk] #7
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -68,20 +68,20 @@ WholeStageCodegen (16)
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #10
+ CometBroadcastExchange [t_time_sk] #10
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -93,20 +93,20 @@ WholeStageCodegen (16)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #13
+ CometBroadcastExchange [t_time_sk] #13
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -118,20 +118,20 @@ WholeStageCodegen (16)
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #16
+ CometBroadcastExchange [t_time_sk] #16
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -143,20 +143,20 @@ WholeStageCodegen (16)
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #19
+ CometBroadcastExchange [t_time_sk] #19
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -168,20 +168,20 @@ WholeStageCodegen (16)
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #22
+ CometBroadcastExchange [t_time_sk] #22
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
InputAdapter
@@ -193,19 +193,19 @@ WholeStageCodegen (16)
WholeStageCodegen (14)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #25
+ CometBroadcastExchange [t_time_sk] #25
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
ReusedExchange [s_store_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt
index 507ac8a91..aa858c158 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt
@@ -15,17 +15,17 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price]
+ CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price]
CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy]
CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_category,i_class,i_item_sk]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -33,12 +33,12 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk,d_moy] #5
CometProject [d_date_sk,d_moy]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
+ CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt
index c54606f6e..5cb600551 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt
@@ -9,9 +9,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #1
ReusedSubquery [mergedValue] #1
@@ -24,9 +24,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #2
ReusedSubquery [mergedValue] #2
@@ -39,9 +39,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #3
ReusedSubquery [mergedValue] #3
@@ -54,9 +54,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #4
ReusedSubquery [mergedValue] #4
@@ -69,9 +69,9 @@ WholeStageCodegen (1)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ext_discount_amt,ss_net_paid]
+ CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid]
CometProject [ss_ext_discount_amt,ss_net_paid]
- CometFilter [ss_quantity]
+ CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk]
ReusedSubquery [mergedValue] #5
ReusedSubquery [mergedValue] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt
index 50c8494fb..44159cc73 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt
@@ -7,27 +7,27 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
CometProject [ws_web_page_sk]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk]
CometProject [ws_sold_time_sk,ws_web_page_sk]
- CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk]
CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
- CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
+ CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_time_sk]
+ CometFilter [t_time_sk,t_hour]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour]
- CometBroadcastExchange #4
+ CometBroadcastExchange [wp_web_page_sk] #4
CometProject [wp_web_page_sk]
- CometFilter [wp_char_count,wp_web_page_sk]
+ CometFilter [wp_web_page_sk,wp_char_count]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count]
InputAdapter
BroadcastExchange #5
@@ -38,19 +38,19 @@ WholeStageCodegen (4)
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
CometProject [ws_web_page_sk]
- CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk]
CometProject [ws_sold_time_sk,ws_web_page_sk]
- CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk]
CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk]
- CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk]
+ CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk]
ReusedExchange [hd_demo_sk] #2
- CometBroadcastExchange #7
+ CometBroadcastExchange [t_time_sk] #7
CometProject [t_time_sk]
- CometFilter [t_hour,t_time_sk]
+ CometFilter [t_time_sk,t_hour]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour]
ReusedExchange [wp_web_page_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt
index b415eb5c4..232c174cd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt
@@ -9,23 +9,23 @@ WholeStageCodegen (3)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss]
+ CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk]
- CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk]
- CometFilter [cc_call_center_sk]
+ CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
+ CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager]
- CometBroadcastExchange #3
- CometFilter [cr_call_center_sk,cr_returning_customer_sk]
+ CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3
+ CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -33,23 +33,23 @@ WholeStageCodegen (3)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
- CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
- CometBroadcastExchange #7
+ CometBroadcastExchange [ca_address_sk] #7
CometProject [ca_address_sk]
- CometFilter [ca_gmt_offset,ca_address_sk]
+ CometFilter [ca_address_sk,ca_gmt_offset]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset]
- CometBroadcastExchange #8
- CometFilter [cd_marital_status,cd_education_status,cd_demo_sk]
+ CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8
+ 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 #9
+ CometBroadcastExchange [hd_demo_sk] #9
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt
index e2f498028..0681a64bd 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt
@@ -11,8 +11,8 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk,ws_ext_discount_amt]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,i_item_sk]
+ CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -20,11 +20,11 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #3
+ CometBroadcastExchange [i_item_sk] #3
CometProject [i_item_sk]
- CometFilter [i_manufact_id,i_item_sk]
+ CometFilter [i_item_sk,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id]
InputAdapter
BroadcastExchange #4
@@ -36,15 +36,15 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,ws_ext_discount_amt]
+ CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt]
CometProject [ws_item_sk,ws_ext_discount_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt
index 3ec7ac7b6..a8eb231b7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt
@@ -28,7 +28,7 @@ TakeOrderedAndProject [sumsales,ss_customer_sk]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity]
- CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk]
+ CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk]
InputAdapter
BroadcastExchange #4
@@ -36,5 +36,5 @@ TakeOrderedAndProject [sumsales,ss_customer_sk]
ColumnarToRow
InputAdapter
CometProject [r_reason_sk]
- CometFilter [r_reason_desc,r_reason_sk]
+ CometFilter [r_reason_sk,r_reason_desc]
CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt
index 34ddde768..d54c9e0c9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt
@@ -26,7 +26,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit]
- CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk]
+ CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (4)
@@ -54,7 +54,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -62,7 +62,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -70,5 +70,5 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [web_site_sk]
- CometFilter [web_company_name,web_site_sk]
+ CometFilter [web_site_sk,web_company_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt
index 5b699890c..178c25f66 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt
@@ -25,7 +25,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit]
- CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk]
+ CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (7)
@@ -40,7 +40,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ws_warehouse_sk,ws_order_number]
- CometFilter [ws_order_number,ws_warehouse_sk]
+ CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -60,7 +60,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [wr_order_number]
- CometFilter [wr_order_number]
+ CometFilter [wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (15)
@@ -82,7 +82,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #6
@@ -90,7 +90,7 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #7
@@ -98,5 +98,5 @@ WholeStageCodegen (21)
ColumnarToRow
InputAdapter
CometProject [web_site_sk]
- CometFilter [web_company_name,web_site_sk]
+ CometFilter [web_site_sk,web_company_name]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt
index 614915226..e9d33a7f6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt
@@ -5,25 +5,25 @@ WholeStageCodegen (2)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate
+ CometHashAggregate [count]
CometProject
CometBroadcastHashJoin [ss_store_sk,s_store_sk]
CometProject [ss_store_sk]
- CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk]
CometProject [ss_sold_time_sk,ss_store_sk]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk]
CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk]
- CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk]
+ CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk]
- CometBroadcastExchange #2
+ CometBroadcastExchange [hd_demo_sk] #2
CometProject [hd_demo_sk]
- CometFilter [hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_dep_count]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count]
- CometBroadcastExchange #3
+ CometBroadcastExchange [t_time_sk] #3
CometProject [t_time_sk]
- CometFilter [t_hour,t_minute,t_time_sk]
+ CometFilter [t_time_sk,t_hour,t_minute]
CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute]
- CometBroadcastExchange #4
+ CometBroadcastExchange [s_store_sk] #4
CometProject [s_store_sk]
- CometFilter [s_store_name,s_store_sk]
+ CometFilter [s_store_sk,s_store_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt
index 24e6dceef..41f3e579f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt
@@ -17,7 +17,7 @@ WholeStageCodegen (6)
InputAdapter
CometHashAggregate [ss_customer_sk,ss_item_sk]
CometProject [ss_item_sk,ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -25,11 +25,11 @@ WholeStageCodegen (6)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
WholeStageCodegen (4)
@@ -42,7 +42,7 @@ WholeStageCodegen (6)
InputAdapter
CometHashAggregate [cs_bill_customer_sk,cs_item_sk]
CometProject [cs_bill_customer_sk,cs_item_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #4
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt
index c03b8be9a..3895cdc78 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt
@@ -18,12 +18,12 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -31,12 +31,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
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/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt
index adfe90bab..fc63929bb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt
@@ -6,27 +6,27 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk]
+ CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk]
CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression]
- CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name,d_date_sk]
CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name]
CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type]
- CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type]
CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk]
- CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name]
+ CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk]
- CometBroadcastExchange #2
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #3
- CometFilter [sm_ship_mode_sk]
+ CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3
+ CometFilter [sm_ship_mode_sk,sm_type]
CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type]
- CometBroadcastExchange #4
- CometFilter [cc_call_center_sk]
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #4
+ CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt
index 520edc88d..22c208f2a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt
@@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [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]
+ CometHashAggregate [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]
CometProject [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]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,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]
CometProject [c_current_cdemo_sk]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk]
CometProject [c_current_cdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [c_customer_sk,customer_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometUnion
+ CometBroadcastExchange [customer_sk] #5
+ CometUnion [customer_sk]
CometProject [ws_bill_customer_sk] [customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
CometProject [cs_ship_customer_sk] [customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk] #6
CometProject [ca_address_sk]
- CometFilter [ca_county,ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
- CometBroadcastExchange #7
- CometFilter [cd_demo_sk]
+ CometBroadcastExchange [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] #7
+ CometFilter [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]
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-v2_7-spark3_5/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt
index ecc421bd5..25010c5eb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt
@@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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
@@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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-v2_7-spark3_5/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt
index 545f0ecec..a8b8a9cce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt
index 799f74a36..aa49638c7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt
@@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #2
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
@@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ 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
BroadcastExchange #4
@@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #3
BroadcastExchange #6
@@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ 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 #10
+ CometBroadcastExchange [d_date_sk] #10
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #10
InputAdapter
@@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #6
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt
index 45061c290..7ef901a39 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt
@@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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] #12
CometProject [cs_quantity,cs_list_price] [quantity,list_price]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #4
BroadcastExchange #16
@@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #17
+ CometBroadcastExchange [d_date_sk] #17
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [ws_quantity,ws_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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] #4
ReusedExchange [d_date_sk] #17
@@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #4
@@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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]
+ 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
BroadcastExchange #6
@@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
@@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ CometBroadcastHashJoin [i_item_sk,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 #10
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
- CometBroadcastExchange #11
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
+ 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 #12
+ CometBroadcastExchange [d_date_sk] #12
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #12
InputAdapter
@@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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] #11
@@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [i_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ 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] #5
@@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [cs_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [ws_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk]
+ CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt
index d4deedf0a..f854b2b31 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt
@@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [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,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_county,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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
@@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3
CometProject [cd_demo_sk,cd_dep_count]
- CometFilter [cd_gender,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #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]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year]
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
+ CometBroadcastExchange [cd_demo_sk] #5
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #6
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6
+ CometFilter [ca_address_sk,ca_county,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #8
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
WholeStageCodegen (4)
HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count]
@@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id]
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_state,ca_country]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_state,ca_country,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_state,ca_country]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #10
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id]
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_country]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_country,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_country]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #12
+ CometBroadcastExchange [ca_address_sk,ca_country] #12
CometProject [ca_address_sk,ca_country]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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,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]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #14
+ CometBroadcastExchange [ca_address_sk] #14
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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,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]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
@@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ReusedExchange [cd_demo_sk] #5
ReusedExchange [ca_address_sk] #14
ReusedExchange [d_date_sk] #7
- CometBroadcastExchange #16
+ CometBroadcastExchange [i_item_sk] #16
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt
index 513e6f979..e460ce5c9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt
index 415b430f8..0e864ab70 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt
@@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [inv_item_sk,inv_quantity_on_hand]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
- CometFilter [inv_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
InputAdapter
BroadcastExchange #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt
index 6ab6c4fd7..30307e528 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt
@@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand]
+ CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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
@@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [w_warehouse_sk] #5
CometFilter [w_warehouse_sk]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk]
WholeStageCodegen (5)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt
index 7024f439f..5b658f1d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt
@@ -43,7 +43,7 @@ WholeStageCodegen (12)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8
@@ -78,7 +78,7 @@ WholeStageCodegen (12)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -89,7 +89,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #6
@@ -97,26 +97,26 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk,ca_country,ca_zip]
+ CometFilter [ca_address_sk,ca_state,ca_zip,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt
index 371254b08..de75d46ee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt
@@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4]
+ CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4]
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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
@@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [s_state,s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #5
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count]
@@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4]
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk] #3
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #8
+ CometBroadcastExchange [s_store_sk] #8
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [i_item_sk,i_item_id] #6
WholeStageCodegen (6)
@@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [agg1,agg2,agg3,agg4]
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4]
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk] #3
ReusedExchange [d_date_sk] #4
ReusedExchange [s_store_sk] #8
- CometBroadcastExchange #10
+ CometBroadcastExchange [i_item_sk] #10
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt
index 970562e5c..814e543ed 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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-v2_7-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt
index a6b4add1f..0b55e23ab 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt
@@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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-v2_7-spark3_5/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt
index e3a91e471..8d73022ee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt
@@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum]
CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
CometProject [c_current_cdemo_sk,ca_state]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state]
CometProject [c_current_cdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [c_customer_sk,customsk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #5
- CometUnion
+ CometBroadcastExchange [customsk] #5
+ CometUnion [customsk]
CometProject [ws_bill_customer_sk] [customsk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
CometProject [cs_ship_customer_sk] [customsk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
- CometBroadcastExchange #6
- CometFilter [ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state] #6
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #7
- CometFilter [cd_demo_sk]
+ CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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-v2_7-spark3_5/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt
index 9c35ee397..256fddfc8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt
@@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price]
+ CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price]
CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category]
CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #6
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #7
+ CometBroadcastExchange [s_store_sk] #7
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
WholeStageCodegen (5)
HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt
index a85302cc5..eeeb5ba36 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price]
+ CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price]
CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt
index 8d7b158d0..0e6b65b06 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
+ CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
- CometBroadcastExchange #4
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
+ CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk]
- CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_return_amt,wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (10)
Project [item,return_ratio,return_rank,currency_rank]
@@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
+ CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
+ CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk]
- CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_return_amount,cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
ReusedExchange [d_date_sk] #6
WholeStageCodegen (15)
@@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
+ CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
- CometBroadcastExchange #12
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
+ CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk]
- CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
ReusedExchange [d_date_sk] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt
index 78dd29c2a..e9c4d46ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt
@@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,d_date,ws_sales_price]
+ CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price]
CometProject [ws_item_sk,ws_sales_price,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #6
@@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
InputAdapter
BroadcastExchange #8
@@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,d_date,ss_sales_price]
+ CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price]
CometProject [ss_item_sk,ss_sales_price,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #7
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt
index c2c8e089f..968f7e1af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price]
+ CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price]
CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [cs_item_sk,cs_call_center_sk]
+ CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3
+ CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #6
CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt
index dd66c9582..1428c7ba5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt
@@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,s_store_id]
- CometBroadcastHashJoin [store_sk,s_store_sk]
+ CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id]
CometProject [store_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ss_store_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [sr_store_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #6
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #6
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
WholeStageCodegen (4)
HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum]
@@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id]
- CometBroadcastHashJoin [page_sk,cp_catalog_page_sk]
+ CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id]
CometProject [page_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cs_catalog_page_sk]
+ CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cr_catalog_page_sk]
+ CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #8
- CometFilter [cp_catalog_page_sk]
+ CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
WholeStageCodegen (6)
HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum]
@@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,web_site_id]
- CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id]
CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ws_web_site_sk]
+ CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number]
- CometBroadcastExchange #10
+ CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number]
+ CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_item_sk,ws_web_site_sk,ws_order_number]
- CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #11
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_site_id] #11
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
WholeStageCodegen (17)
HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt
index 9607ab887..cfb74b3b2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt
@@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state]
ColumnarToRow
InputAdapter
CometProject [ca_state,ss_item_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk]
CometProject [ca_state,ss_item_sk,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometProject [ca_state,c_customer_sk]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #2
- CometFilter [c_current_addr_sk,c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2
+ CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #3
- CometFilter [ss_customer_sk,ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
Subquery #2
WholeStageCodegen (2)
HashAggregate [d_month_seq]
@@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state]
InputAdapter
CometHashAggregate [d_month_seq]
CometProject [d_month_seq]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
@@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state]
BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
ColumnarToRow
InputAdapter
- CometFilter [i_current_price,i_category,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category]
InputAdapter
BroadcastExchange #8
@@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_current_price]
- CometFilter [i_category]
+ CometHashAggregate [i_category,sum,count,i_current_price]
+ CometFilter [i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_current_price,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt
index 2a0bc5bce..512a74f9f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt
@@ -54,19 +54,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #4
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -89,7 +89,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_ext_list_price]
- CometFilter [cs_item_sk,cs_order_number]
+ CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -100,7 +100,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #5
@@ -116,14 +116,14 @@ WholeStageCodegen (52)
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometFilter [d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
ReusedExchange [d_date_sk,d_year] #11
@@ -157,7 +157,7 @@ WholeStageCodegen (52)
WholeStageCodegen (19)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
InputAdapter
ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
@@ -176,7 +176,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_product_name]
- CometFilter [i_current_price,i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_color,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name]
InputAdapter
WholeStageCodegen (50)
@@ -227,19 +227,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #20
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #21
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (33)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt
index aeb5b67da..5838786d4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt
@@ -20,14 +20,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity]
+ CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity]
CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy]
+ CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
WholeStageCodegen (5)
HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt
index 74305dd6e..e76a524db 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt
@@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #6
@@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
BroadcastHashJoin [s_state,s_state]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_county,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state]
InputAdapter
BroadcastExchange #7
@@ -65,16 +65,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_state,ss_net_profit]
+ CometHashAggregate [s_state,sum,ss_net_profit]
CometProject [ss_net_profit,s_state]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk]
CometProject [ss_net_profit,ss_sold_date_sk,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #9
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #9
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [d_date_sk] #5
WholeStageCodegen (13)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt
index bea1fd4a1..6cba2d0e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt
@@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk]
- CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity]
- CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #4
- CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
+ CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #6
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
- CometBroadcastExchange #7
+ CometBroadcastExchange [cd_demo_sk] #7
CometProject [cd_demo_sk]
- CometFilter [cd_marital_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_marital_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status]
- CometBroadcastExchange #8
+ CometBroadcastExchange [hd_demo_sk] #8
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #10
- CometFilter [d_week_seq,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_week_seq] #10
+ CometFilter [d_date_sk,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
@@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt
index 283fe5bde..e5cf23f50 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt
@@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #2
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #7
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #12
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #15
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,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-v2_7-spark3_5/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt
index fb78d64b1..2aecc9e98 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt
@@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
- CometBroadcastExchange #7
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #7
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
WholeStageCodegen (10)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
@@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
WholeStageCodegen (15)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
@@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (38)
@@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #17
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
- CometBroadcastExchange #18
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #18
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (23)
@@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt
index 752b8c854..34972d290 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt
@@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit]
+ CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit]
CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
InputAdapter
@@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss]
+ CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss]
CometProject [sr_return_amt,sr_net_loss,s_store_sk]
- CometBroadcastHashJoin [sr_store_sk,s_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk]
CometProject [sr_store_sk,sr_return_amt,sr_net_loss]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- CometFilter [sr_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
+ CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit]
CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_return_amount,cr_net_loss]
+ CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss]
CometProject [cr_return_amount,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit]
+ CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit]
CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #13
+ CometBroadcastExchange [wp_web_page_sk] #13
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
InputAdapter
@@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss]
+ CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss]
CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk]
- CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk]
CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
- CometFilter [wr_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt
index 49bd173f6..b8e6e0605 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk,ss_customer_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt
index 34e47dcba..0793067df 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt
@@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
InputAdapter
BroadcastExchange #8
@@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [i_item_sk]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price]
InputAdapter
BroadcastExchange #9
@@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [p_promo_sk]
- CometFilter [p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv]
WholeStageCodegen (20)
HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty]
@@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk]
+ CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cp_catalog_page_sk]
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
InputAdapter
ReusedExchange [i_item_sk] #8
@@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (21)
ColumnarToRow
InputAdapter
- CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss]
- CometFilter [wr_item_sk,wr_order_number]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (26)
ColumnarToRow
InputAdapter
- CometFilter [web_site_sk]
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
InputAdapter
ReusedExchange [i_item_sk] #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt
index 0b7ad4726..5adee59be 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt
@@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,ws_net_paid]
+ CometHashAggregate [i_category,i_class,sum,ws_net_paid]
CometProject [ws_net_paid,i_class,i_category]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category]
CometProject [ws_item_sk,ws_net_paid]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #6
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
WholeStageCodegen (5)
HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt
index 6484c2dcb..64b162608 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt
@@ -17,12 +17,12 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,12 +30,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt
index 520edc88d..22c208f2a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt
@@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [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]
+ CometHashAggregate [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]
CometProject [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]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,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]
CometProject [c_current_cdemo_sk]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk]
CometProject [c_current_cdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [c_customer_sk,customer_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometUnion
+ CometBroadcastExchange [customer_sk] #5
+ CometUnion [customer_sk]
CometProject [ws_bill_customer_sk] [customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
CometProject [cs_ship_customer_sk] [customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk] #6
CometProject [ca_address_sk]
- CometFilter [ca_county,ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
- CometBroadcastExchange #7
- CometFilter [cd_demo_sk]
+ CometBroadcastExchange [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] #7
+ CometFilter [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]
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-v2_7-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt
index 4b9e60e18..d7c6ef3b2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt
@@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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
@@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt
index 3d0576653..07c91b94d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt
@@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt
index 73badab8d..cd4817202 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt
@@ -11,19 +11,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
HashAggregate [quantity,list_price] [sum,count,sum,count]
ColumnarToRow
InputAdapter
- CometUnion
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -47,7 +47,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
ReusedSubquery [d_week_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
Subquery #2
@@ -55,7 +55,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,d_year,d_moy,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
InputAdapter
BroadcastExchange #3
@@ -64,7 +64,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ 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
BroadcastExchange #4
@@ -78,10 +78,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #3
BroadcastExchange #6
@@ -89,26 +89,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ 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 #10
+ CometBroadcastExchange [d_date_sk] #10
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #10
InputAdapter
@@ -117,10 +117,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -154,7 +154,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -162,7 +162,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
ReusedSubquery [d_week_seq] #6
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
Subquery #6
@@ -170,7 +170,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,d_year,d_moy,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom]
InputAdapter
ReusedExchange [ss_item_sk] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt
index 6e0ac9858..3dbd7bc33 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt
@@ -26,14 +26,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
HashAggregate [quantity,list_price] [sum,count,sum,count]
ColumnarToRow
InputAdapter
- CometUnion
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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] #12
CometProject [cs_quantity,cs_list_price] [quantity,list_price]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #4
BroadcastExchange #16
@@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #17
+ CometBroadcastExchange [d_date_sk] #17
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [ws_quantity,ws_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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] #4
ReusedExchange [d_date_sk] #17
@@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #4
@@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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]
+ 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
BroadcastExchange #6
@@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
@@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ CometBroadcastHashJoin [i_item_sk,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 #10
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
- CometBroadcastExchange #11
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
+ 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 #12
+ CometBroadcastExchange [d_date_sk] #12
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #12
InputAdapter
@@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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] #11
@@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [i_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ 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] #5
@@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [cs_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [ws_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk]
+ CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt
index 9f43cdc72..509d4be06 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt
@@ -9,18 +9,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [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,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_county,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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
@@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3
CometProject [cd_demo_sk,cd_dep_count]
- CometFilter [cd_gender,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #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]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year]
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
+ CometBroadcastExchange [cd_demo_sk] #5
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #6
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6
+ CometFilter [ca_address_sk,ca_county,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #8
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
WholeStageCodegen (4)
HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count]
@@ -60,25 +60,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id]
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_state,ca_country]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_state,ca_country,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_state,ca_country]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #10
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -91,26 +91,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id]
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_country]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_country,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_country]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #12
+ CometBroadcastExchange [ca_address_sk,ca_country] #12
CometProject [ca_address_sk,ca_country]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -123,26 +123,26 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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,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]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #14
+ CometBroadcastExchange [ca_address_sk] #14
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -155,18 +155,18 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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,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]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
@@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ReusedExchange [cd_demo_sk] #5
ReusedExchange [ca_address_sk] #14
ReusedExchange [d_date_sk] #7
- CometBroadcastExchange #16
+ CometBroadcastExchange [i_item_sk] #16
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt
index da567f687..cd6bcfd61 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt
@@ -16,10 +16,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
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]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt
index 415b430f8..0e864ab70 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt
@@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [inv_item_sk,inv_quantity_on_hand]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
- CometFilter [inv_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
InputAdapter
BroadcastExchange #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt
index db81d1ba5..fc4778a22 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt
@@ -11,12 +11,12 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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
@@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [w_warehouse_sk] #5
CometFilter [w_warehouse_sk]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk]
WholeStageCodegen (5)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt
index 7024f439f..5b658f1d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt
@@ -43,7 +43,7 @@ WholeStageCodegen (12)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8
@@ -78,7 +78,7 @@ WholeStageCodegen (12)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -89,7 +89,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #6
@@ -97,26 +97,26 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk,ca_country,ca_zip]
+ CometFilter [ca_address_sk,ca_state,ca_zip,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt
index b09f5af82..50e246e15 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt
@@ -9,14 +9,14 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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
@@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [s_state,s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #5
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count]
@@ -49,21 +49,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk] #3
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #8
+ CometBroadcastExchange [s_store_sk] #8
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [i_item_sk,i_item_id] #6
WholeStageCodegen (6)
@@ -75,19 +75,19 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk] #3
ReusedExchange [d_date_sk] #4
ReusedExchange [s_store_sk] #8
- CometBroadcastExchange #10
+ CometBroadcastExchange [i_item_sk] #10
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt
index 970562e5c..814e543ed 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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-v2_7-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt
index a6b4add1f..0b55e23ab 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt
@@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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-v2_7-spark4_0/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt
index a95475f7b..79ead0464 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt
@@ -8,17 +8,17 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
CometProject [c_current_cdemo_sk,ca_state]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state]
CometProject [c_current_cdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [c_customer_sk,customsk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #5
- CometUnion
+ CometBroadcastExchange [customsk] #5
+ CometUnion [customsk]
CometProject [ws_bill_customer_sk] [customsk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
CometProject [cs_ship_customer_sk] [customsk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
- CometBroadcastExchange #6
- CometFilter [ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state] #6
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #7
- CometFilter [cd_demo_sk]
+ CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt
index 485fdc3e2..59c22c335 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt
@@ -24,12 +24,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category]
CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #6
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #7
+ CometBroadcastExchange [s_store_sk] #7
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
WholeStageCodegen (5)
HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt
index a480db5ea..61f4976b4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt
@@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,
ColumnarToRow
InputAdapter
CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt
index d1480ef10..bd3b21cdb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt
@@ -29,12 +29,12 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
- CometBroadcastExchange #4
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
+ CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk]
- CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_return_amt,wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (10)
Project [item,return_ratio,return_rank,currency_rank]
@@ -73,16 +73,16 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
+ CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk]
- CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_return_amount,cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
ReusedExchange [d_date_sk] #6
WholeStageCodegen (15)
@@ -107,15 +107,15 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
- CometBroadcastExchange #12
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
+ CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk]
- CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
ReusedExchange [d_date_sk] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt
index 892a3fc78..47ad5e34f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt
@@ -44,8 +44,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_sales_price,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #6
@@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
InputAdapter
BroadcastExchange #8
@@ -98,8 +98,8 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_sales_price,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #7
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt
index b038aedf0..daeafbf92 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt
@@ -25,27 +25,27 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m
ColumnarToRow
InputAdapter
CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [cs_item_sk,cs_call_center_sk]
+ CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3
+ CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #6
CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt
index 34ad132c0..c3b00538b 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt
@@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sales_price,profit,return_amt,net_loss,s_store_id]
- CometBroadcastHashJoin [store_sk,s_store_sk]
+ CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id]
CometProject [store_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ss_store_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [sr_store_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #6
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #6
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
WholeStageCodegen (4)
HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum]
@@ -59,21 +59,21 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id]
- CometBroadcastHashJoin [page_sk,cp_catalog_page_sk]
+ CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id]
CometProject [page_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cs_catalog_page_sk]
+ CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cr_catalog_page_sk]
+ CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #8
- CometFilter [cp_catalog_page_sk]
+ CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
WholeStageCodegen (6)
HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum]
@@ -84,25 +84,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sales_price,profit,return_amt,net_loss,web_site_id]
- CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id]
CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ws_web_site_sk]
+ CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number]
- CometBroadcastExchange #10
+ CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number]
+ CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_item_sk,ws_web_site_sk,ws_order_number]
- CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #11
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_site_id] #11
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
WholeStageCodegen (17)
HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt
index a7c65f50a..febbfd32f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt
@@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [ca_state,ss_item_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk]
CometProject [ca_state,ss_item_sk,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometProject [ca_state,c_customer_sk]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #2
- CometFilter [c_current_addr_sk,c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2
+ CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #3
- CometFilter [ss_customer_sk,ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,state]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
Subquery #2
@@ -43,11 +43,11 @@ TakeOrderedAndProject [cnt,state]
InputAdapter
CometHashAggregate [d_month_seq]
CometProject [d_month_seq]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
@@ -58,7 +58,7 @@ TakeOrderedAndProject [cnt,state]
BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
ColumnarToRow
InputAdapter
- CometFilter [i_current_price,i_category,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category]
InputAdapter
BroadcastExchange #8
@@ -71,5 +71,5 @@ TakeOrderedAndProject [cnt,state]
HashAggregate [i_category,i_current_price] [sum,count,sum,count]
ColumnarToRow
InputAdapter
- CometFilter [i_category]
+ CometFilter [i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_current_price,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt
index 2a0bc5bce..512a74f9f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt
@@ -54,19 +54,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #4
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -89,7 +89,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_ext_list_price]
- CometFilter [cs_item_sk,cs_order_number]
+ CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -100,7 +100,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #5
@@ -116,14 +116,14 @@ WholeStageCodegen (52)
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometFilter [d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
ReusedExchange [d_date_sk,d_year] #11
@@ -157,7 +157,7 @@ WholeStageCodegen (52)
WholeStageCodegen (19)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
InputAdapter
ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
@@ -176,7 +176,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_product_name]
- CometFilter [i_current_price,i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_color,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name]
InputAdapter
WholeStageCodegen (50)
@@ -227,19 +227,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #20
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #21
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (33)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt
index f68ac8dfc..cc9627e89 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt
@@ -22,12 +22,12 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
ColumnarToRow
InputAdapter
CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy]
+ CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -35,17 +35,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
WholeStageCodegen (5)
HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt
index a7c639a49..67f4841b4 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt
@@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #6
@@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
BroadcastHashJoin [s_state,s_state]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_county,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state]
InputAdapter
BroadcastExchange #7
@@ -67,14 +67,14 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_net_profit,s_state]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk]
CometProject [ss_net_profit,ss_sold_date_sk,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #9
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #9
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [d_date_sk] #5
WholeStageCodegen (13)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt
index bea1fd4a1..6cba2d0e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt
@@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk]
- CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity]
- CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #4
- CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
+ CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #6
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
- CometBroadcastExchange #7
+ CometBroadcastExchange [cd_demo_sk] #7
CometProject [cd_demo_sk]
- CometFilter [cd_marital_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_marital_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status]
- CometBroadcastExchange #8
+ CometBroadcastExchange [hd_demo_sk] #8
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #10
- CometFilter [d_week_seq,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_week_seq] #10
+ CometFilter [d_date_sk,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
@@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt
index a2cb15b4e..4360a9555 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt
@@ -14,23 +14,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #2
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -43,23 +43,23 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #7
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -73,13 +73,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #12
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -94,13 +94,13 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
ColumnarToRow
InputAdapter
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #15
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,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-v2_7-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt
index fb78d64b1..2aecc9e98 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt
@@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
- CometBroadcastExchange #7
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #7
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
WholeStageCodegen (10)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
@@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
WholeStageCodegen (15)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
@@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (38)
@@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #17
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
- CometBroadcastExchange #18
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #18
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (23)
@@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt
index 141e72805..db4e10247 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt
@@ -26,10 +26,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
InputAdapter
@@ -57,10 +57,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_return_amt,sr_net_loss,s_store_sk]
- CometBroadcastHashJoin [sr_store_sk,s_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk]
CometProject [sr_store_sk,sr_return_amt,sr_net_loss]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- CometFilter [sr_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -79,7 +79,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -91,7 +91,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_return_amount,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -106,14 +106,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #13
+ CometBroadcastExchange [wp_web_page_sk] #13
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
InputAdapter
@@ -127,10 +127,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk]
- CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk]
CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
- CometFilter [wr_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt
index 49bd173f6..b8e6e0605 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk,ss_customer_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt
index 34e47dcba..0793067df 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt
@@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
InputAdapter
BroadcastExchange #8
@@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [i_item_sk]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price]
InputAdapter
BroadcastExchange #9
@@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [p_promo_sk]
- CometFilter [p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv]
WholeStageCodegen (20)
HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty]
@@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk]
+ CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cp_catalog_page_sk]
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
InputAdapter
ReusedExchange [i_item_sk] #8
@@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (21)
ColumnarToRow
InputAdapter
- CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss]
- CometFilter [wr_item_sk,wr_order_number]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (26)
ColumnarToRow
InputAdapter
- CometFilter [web_site_sk]
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
InputAdapter
ReusedExchange [i_item_sk] #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt
index 7edc6b80b..8996119f2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt
@@ -24,10 +24,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
ColumnarToRow
InputAdapter
CometProject [ws_net_paid,i_class,i_category]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category]
CometProject [ws_item_sk,ws_net_paid]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #6
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
WholeStageCodegen (5)
HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt
index 2dd86f85b..3fc9eb261 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt
@@ -19,10 +19,10 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,12 +30,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt
index 520edc88d..22c208f2a 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt
@@ -6,19 +6,19 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [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]
+ CometHashAggregate [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]
CometProject [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]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,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]
CometProject [c_current_cdemo_sk]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk]
CometProject [c_current_cdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [c_customer_sk,customer_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customer_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -26,28 +26,28 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometUnion
+ CometBroadcastExchange [customer_sk] #5
+ CometUnion [customer_sk]
CometProject [ws_bill_customer_sk] [customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
CometProject [cs_ship_customer_sk] [customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
- CometBroadcastExchange #6
+ CometBroadcastExchange [ca_address_sk] #6
CometProject [ca_address_sk]
- CometFilter [ca_county,ca_address_sk]
+ CometFilter [ca_address_sk,ca_county]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county]
- CometBroadcastExchange #7
- CometFilter [cd_demo_sk]
+ CometBroadcastExchange [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] #7
+ CometFilter [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]
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-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt
index ecc421bd5..25010c5eb 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt
@@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_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]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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
@@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
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]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,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]
+ CometBroadcastHashJoin [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,d_date_sk,d_year]
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]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address]
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]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_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-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt
index 545f0ecec..a8b8a9cce 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt
index 799f74a36..aa49638c7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt
@@ -10,20 +10,20 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,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]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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
@@ -39,7 +39,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -47,13 +47,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #2
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
@@ -63,7 +63,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ 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
BroadcastExchange #4
@@ -77,10 +77,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #3
BroadcastExchange #6
@@ -88,26 +88,26 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7
+ CometBroadcastHashJoin [i_item_sk,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
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #8
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_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]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ 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 #10
+ CometBroadcastExchange [d_date_sk] #10
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #10
InputAdapter
@@ -116,10 +116,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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
@@ -153,7 +153,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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
@@ -161,13 +161,13 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_week_seq,d_date_sk]
+ CometFilter [d_date_sk,d_week_seq]
Subquery #6
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
CometProject [d_week_seq]
- CometFilter [d_year,d_moy,d_dom]
+ CometFilter [d_week_seq,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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt
index 45061c290..7ef901a39 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt
@@ -25,15 +25,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [quantity,list_price]
- CometUnion
+ CometHashAggregate [sum,count,quantity,list_price]
+ CometUnion [quantity,list_price]
CometProject [ss_quantity,ss_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_quantity,ss_list_price,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] #12
CometProject [cs_quantity,cs_list_price] [quantity,list_price]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #4
BroadcastExchange #16
@@ -41,14 +41,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #17
+ CometBroadcastExchange [d_date_sk] #17
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [ws_quantity,ws_list_price] [quantity,list_price]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_quantity,ws_list_price,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] #4
ReusedExchange [d_date_sk] #17
@@ -64,7 +64,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [ss_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk]
+ CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_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 #4
@@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
InputAdapter
BroadcastExchange #5
@@ -81,7 +81,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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]
+ 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
BroadcastExchange #6
@@ -95,10 +95,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
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]
+ CometBroadcastHashJoin [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ss_item_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
@@ -106,26 +106,26 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9
+ CometBroadcastHashJoin [i_item_sk,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 #10
+ CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #10
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [cs_item_sk,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #2
- CometBroadcastExchange #11
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11
+ 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 #12
+ CometBroadcastExchange [d_date_sk] #12
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [d_date_sk] #12
InputAdapter
@@ -134,10 +134,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
ColumnarToRow
InputAdapter
CometProject [i_brand_id,i_class_id,i_category_id]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,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]
+ CometBroadcastHashJoin [ws_item_sk,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id]
+ CometFilter [ws_item_sk,ws_sold_date_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] #11
@@ -148,7 +148,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [i_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ 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] #5
@@ -169,7 +169,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [cs_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk]
+ CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_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
@@ -193,7 +193,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
BroadcastHashJoin [ws_item_sk,ss_item_sk]
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk]
+ CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_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
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt
index d4deedf0a..f854b2b31 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt
@@ -7,20 +7,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [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,i_item_sk,i_item_id]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_county,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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
@@ -28,28 +28,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3
CometProject [cd_demo_sk,cd_dep_count]
- CometFilter [cd_gender,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count]
- CometBroadcastExchange #4
+ CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #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]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year]
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
+ CometBroadcastExchange [cd_demo_sk] #5
CometFilter [cd_demo_sk]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk]
- CometBroadcastExchange #6
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6
+ CometFilter [ca_address_sk,ca_county,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk] #7
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #8
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #8
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
WholeStageCodegen (4)
HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count]
@@ -58,27 +58,27 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country,i_item_sk,i_item_id]
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_state,ca_country]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_state,ca_country,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_state,ca_country]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_state,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #10
- CometFilter [ca_state,ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state,ca_country] #10
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -89,28 +89,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country,i_item_sk,i_item_id]
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_country]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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_country,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_country]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,ca_address_sk,ca_country]
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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #12
+ CometBroadcastExchange [ca_address_sk,ca_country] #12
CometProject [ca_address_sk,ca_country]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -121,28 +121,28 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_sk,i_item_id]
CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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,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]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4
ReusedExchange [cd_demo_sk] #5
- CometBroadcastExchange #14
+ CometBroadcastExchange [ca_address_sk] #14
CometProject [ca_address_sk]
- CometFilter [ca_state,ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
ReusedExchange [d_date_sk] #7
ReusedExchange [i_item_sk,i_item_id] #8
@@ -153,20 +153,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7]
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7]
CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,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]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [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,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]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,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]
+ CometBroadcastHashJoin [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,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year]
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]
+ CometBroadcastHashJoin [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,cd_demo_sk,cd_dep_count]
+ CometFilter [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]
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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk,cd_dep_count] #3
@@ -174,6 +174,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag
ReusedExchange [cd_demo_sk] #5
ReusedExchange [ca_address_sk] #14
ReusedExchange [d_date_sk] #7
- CometBroadcastExchange #16
+ CometBroadcastExchange [i_item_sk] #16
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt
index 513e6f979..e460ce5c9 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt
@@ -14,12 +14,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,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]
+ CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,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]
+ CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_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
@@ -27,12 +27,12 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #4
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt
index 415b430f8..0e864ab70 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt
@@ -11,10 +11,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [inv_item_sk,i_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [inv_item_sk,inv_quantity_on_hand]
- CometBroadcastHashJoin [inv_date_sk,d_date_sk]
- CometFilter [inv_item_sk]
+ CometBroadcastHashJoin [inv_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #2
@@ -22,14 +22,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
InputAdapter
BroadcastExchange #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt
index 6ab6c4fd7..30307e528 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt
@@ -9,14 +9,14 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand]
+ CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand]
CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name,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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,i_item_sk,i_brand,i_class,i_category,i_product_name]
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]
+ CometBroadcastHashJoin [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk]
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_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
@@ -24,16 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #3
+ CometBroadcastExchange [d_date_sk] #3
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #4
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
- CometBroadcastExchange #5
+ CometBroadcastExchange [w_warehouse_sk] #5
CometFilter [w_warehouse_sk]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk]
WholeStageCodegen (5)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt
index 7024f439f..5b658f1d7 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt
@@ -43,7 +43,7 @@ WholeStageCodegen (12)
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
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_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8
@@ -78,7 +78,7 @@ WholeStageCodegen (12)
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]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_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)
@@ -89,7 +89,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
BroadcastExchange #6
@@ -97,26 +97,26 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
CometProject [s_store_sk,s_store_name,s_state,s_zip]
- CometFilter [s_market_id,s_store_sk,s_zip]
+ CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip]
InputAdapter
BroadcastExchange #7
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country]
+ CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country]
InputAdapter
BroadcastExchange #9
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk,ca_country,ca_zip]
+ CometFilter [ca_address_sk,ca_state,ca_zip,ca_country]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt
index 371254b08..de75d46ee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt
@@ -7,16 +7,16 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4]
+ CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4]
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_store_sk,s_state]
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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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
@@ -24,21 +24,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #3
+ CometBroadcastExchange [cd_demo_sk] #3
CometProject [cd_demo_sk]
- CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
- CometFilter [s_state,s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #5
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id] #6
+ CometFilter [i_item_sk,i_item_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id]
WholeStageCodegen (4)
HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count]
@@ -47,23 +47,23 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4]
+ CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4]
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk,i_item_id]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk] #3
ReusedExchange [d_date_sk] #4
- CometBroadcastExchange #8
+ CometBroadcastExchange [s_store_sk] #8
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [i_item_sk,i_item_id] #6
WholeStageCodegen (6)
@@ -73,21 +73,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [agg1,agg2,agg3,agg4]
+ CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4]
CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_sk]
CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,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]
+ CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk,cd_demo_sk]
+ CometFilter [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_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]
ReusedSubquery [d_date_sk] #1
ReusedExchange [cd_demo_sk] #3
ReusedExchange [d_date_sk] #4
ReusedExchange [s_store_sk] #8
- CometBroadcastExchange #10
+ CometBroadcastExchange [i_item_sk] #10
CometFilter [i_item_sk]
CometScan parquet spark_catalog.default.item [i_item_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt
index 970562e5c..814e543ed 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt
@@ -12,14 +12,14 @@ WholeStageCodegen (4)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_ticket_number,ss_customer_sk]
+ CometHashAggregate [ss_ticket_number,ss_customer_sk,count]
CometProject [ss_customer_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_ticket_number,hd_demo_sk]
CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,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]
+ CometBroadcastHashJoin [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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
@@ -27,24 +27,24 @@ WholeStageCodegen (4)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_dom,d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_dom]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom]
- CometBroadcastExchange #5
+ CometBroadcastExchange [s_store_sk] #5
CometProject [s_store_sk]
- CometFilter [s_county,s_store_sk]
+ CometFilter [s_store_sk,s_county]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county]
- CometBroadcastExchange #6
+ CometBroadcastExchange [hd_demo_sk] #6
CometProject [hd_demo_sk]
- CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count]
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]
+ CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag]
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-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt
index a6b4add1f..0b55e23ab 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt
@@ -15,12 +15,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
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]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -28,11 +28,11 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
InputAdapter
BroadcastExchange #5
@@ -40,7 +40,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [ws_bill_customer_sk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
@@ -50,7 +50,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [cs_ship_customer_sk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
@@ -59,12 +59,12 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
InputAdapter
BroadcastExchange #8
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometFilter [cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt
index e3a91e471..8d73022ee 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt
@@ -6,19 +6,19 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
+ CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum]
CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
- CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,ca_state,cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
CometProject [c_current_cdemo_sk,ca_state]
- CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk]
+ CometBroadcastHashJoin [c_current_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state]
CometProject [c_current_cdemo_sk,c_current_addr_sk]
- CometBroadcastHashJoin [c_customer_sk,customsk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_current_addr_sk,c_current_cdemo_sk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,customsk]
+ CometBroadcastHashJoin [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,ss_customer_sk]
+ CometFilter [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]
- CometBroadcastExchange #2
+ CometBroadcastExchange [ss_customer_sk] #2
CometProject [ss_customer_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_customer_sk,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
@@ -26,27 +26,27 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk] #4
CometProject [d_date_sk]
- CometFilter [d_year,d_qoy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy]
- CometBroadcastExchange #5
- CometUnion
+ CometBroadcastExchange [customsk] #5
+ CometUnion [customsk]
CometProject [ws_bill_customer_sk] [customsk]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_bill_customer_sk,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
CometProject [cs_ship_customer_sk] [customsk]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_customer_sk,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
- CometBroadcastExchange #6
- CometFilter [ca_address_sk]
+ CometBroadcastExchange [ca_address_sk,ca_state] #6
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #7
- CometFilter [cd_demo_sk]
+ CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7
+ CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count]
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-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt
index 9c35ee397..256fddfc8 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt
@@ -22,14 +22,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price]
+ CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price]
CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category]
CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,18 +37,18 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #6
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
- CometBroadcastExchange #7
+ CometBroadcastExchange [s_store_sk] #7
CometProject [s_store_sk]
- CometFilter [s_state,s_store_sk]
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
WholeStageCodegen (5)
HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt
index a85302cc5..eeeb5ba36 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price]
+ CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price]
CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,ss_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [ss_item_sk,ss_store_sk]
+ CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6
CometFilter [s_store_sk,s_store_name,s_company_name]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt
index 8d7b158d0..0e6b65b06 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
+ CometHashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk]
CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt]
- CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
- CometBroadcastExchange #4
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
+ CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk]
- CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -42,14 +42,14 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_return_amt,wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
WholeStageCodegen (10)
Project [item,return_ratio,return_rank,currency_rank]
@@ -71,18 +71,18 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
+ CometHashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk]
CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount]
- CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
- CometBroadcastExchange #9
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
+ CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk]
- CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_return_amount,cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
ReusedExchange [d_date_sk] #6
WholeStageCodegen (15)
@@ -105,17 +105,17 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
+ CometHashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk]
CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt]
- CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
- CometBroadcastExchange #12
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
+ CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk]
- CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
ReusedExchange [d_date_sk] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt
index 78dd29c2a..e9c4d46ca 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt
@@ -42,10 +42,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [ws_item_sk,d_date,ws_sales_price]
+ CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price]
CometProject [ws_item_sk,ws_sales_price,d_date]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #6
@@ -53,11 +53,11 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
- CometBroadcastExchange #7
+ CometBroadcastExchange [d_date_sk,d_date] #7
CometProject [d_date_sk,d_date]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_date,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq]
InputAdapter
BroadcastExchange #8
@@ -96,10 +96,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometHashAggregate [ss_item_sk,d_date,ss_sales_price]
+ CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price]
CometProject [ss_item_sk,ss_sales_price,d_date]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date]
+ CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_date] #7
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt
index c2c8e089f..968f7e1af 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt
@@ -23,29 +23,29 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price]
+ CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price]
CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name]
- CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy]
CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk]
- CometBroadcastHashJoin [i_item_sk,cs_item_sk]
- CometFilter [i_item_sk,i_category,i_brand]
+ CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
+ CometFilter [i_item_sk,i_brand,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category]
- CometBroadcastExchange #3
- CometFilter [cs_item_sk,cs_call_center_sk]
+ CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3
+ CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_moy,d_date_sk]
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #5
- CometFilter [d_year,d_moy,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year,d_moy] #5
+ CometFilter [d_date_sk,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy]
- CometBroadcastExchange #6
+ CometBroadcastExchange [cc_call_center_sk,cc_name] #6
CometFilter [cc_call_center_sk,cc_name]
CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name]
InputAdapter
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt
index dd66c9582..1428c7ba5 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt
@@ -22,14 +22,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,s_store_id]
- CometBroadcastHashJoin [store_sk,s_store_sk]
+ CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id]
CometProject [store_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ss_store_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,18 +37,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [sr_store_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #6
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #6
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
WholeStageCodegen (4)
HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum]
@@ -57,23 +57,23 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (3)
ColumnarToRow
InputAdapter
- CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id]
- CometBroadcastHashJoin [page_sk,cp_catalog_page_sk]
+ CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id]
CometProject [page_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cs_catalog_page_sk]
+ CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [cr_catalog_page_sk]
+ CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #8
- CometFilter [cp_catalog_page_sk]
+ CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
WholeStageCodegen (6)
HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum]
@@ -82,27 +82,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss]
+ CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss]
CometProject [sales_price,profit,return_amt,net_loss,web_site_id]
- CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk]
+ CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id]
CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [date_sk,d_date_sk]
- CometUnion
+ CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk]
+ CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometFilter [ws_web_site_sk]
+ CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss]
- CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number]
- CometBroadcastExchange #10
+ CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number]
+ CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
CometProject [ws_item_sk,ws_web_site_sk,ws_order_number]
- CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk]
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #11
- CometFilter [web_site_sk]
+ CometBroadcastExchange [web_site_sk,web_site_id] #11
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
WholeStageCodegen (17)
HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt
index 9607ab887..cfb74b3b2 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt
@@ -11,18 +11,18 @@ TakeOrderedAndProject [cnt,ca_state,state]
ColumnarToRow
InputAdapter
CometProject [ca_state,ss_item_sk]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ca_state,ss_item_sk,ss_sold_date_sk,d_date_sk]
CometProject [ca_state,ss_item_sk,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
+ CometBroadcastHashJoin [ca_state,c_customer_sk,ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometProject [ca_state,c_customer_sk]
- CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk]
- CometFilter [ca_address_sk]
+ CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk]
+ CometFilter [ca_address_sk,ca_state]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state]
- CometBroadcastExchange #2
- CometFilter [c_current_addr_sk,c_customer_sk]
+ CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2
+ CometFilter [c_customer_sk,c_current_addr_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk]
- CometBroadcastExchange #3
- CometFilter [ss_customer_sk,ss_item_sk]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_sold_date_sk] #3
+ CometFilter [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,7 +30,7 @@ TakeOrderedAndProject [cnt,ca_state,state]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
Subquery #2
WholeStageCodegen (2)
HashAggregate [d_month_seq]
@@ -41,12 +41,12 @@ TakeOrderedAndProject [cnt,ca_state,state]
InputAdapter
CometHashAggregate [d_month_seq]
CometProject [d_month_seq]
- CometFilter [d_year,d_moy]
+ CometFilter [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
ReusedSubquery [d_month_seq] #2
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
@@ -56,7 +56,7 @@ TakeOrderedAndProject [cnt,ca_state,state]
BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
ColumnarToRow
InputAdapter
- CometFilter [i_current_price,i_category,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category]
InputAdapter
BroadcastExchange #8
@@ -68,6 +68,6 @@ TakeOrderedAndProject [cnt,ca_state,state]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_current_price]
- CometFilter [i_category]
+ CometHashAggregate [i_category,sum,count,i_current_price]
+ CometFilter [i_current_price,i_category]
CometScan parquet spark_catalog.default.item [i_current_price,i_category]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt
index 2a0bc5bce..512a74f9f 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt
@@ -54,19 +54,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #4
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #4
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (8)
@@ -89,7 +89,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_ext_list_price]
- CometFilter [cs_item_sk,cs_order_number]
+ CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (6)
@@ -100,7 +100,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #5
@@ -116,14 +116,14 @@ WholeStageCodegen (52)
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk]
+ CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (12)
ColumnarToRow
InputAdapter
- CometFilter [d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
ReusedExchange [d_date_sk,d_year] #11
@@ -157,7 +157,7 @@ WholeStageCodegen (52)
WholeStageCodegen (19)
ColumnarToRow
InputAdapter
- CometFilter [ca_address_sk]
+ CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip]
InputAdapter
ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15
@@ -176,7 +176,7 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [i_item_sk,i_product_name]
- CometFilter [i_current_price,i_color,i_item_sk]
+ CometFilter [i_item_sk,i_current_price,i_color,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name]
InputAdapter
WholeStageCodegen (50)
@@ -227,19 +227,19 @@ WholeStageCodegen (52)
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
- CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number]
- CometBroadcastExchange #20
- CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,sr_item_sk,sr_ticket_number]
+ CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] #20
+ CometFilter [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #21
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
WholeStageCodegen (33)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
index a3a99e312..dfa9ae66d 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
@@ -15,14 +15,14 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity]
+ CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty,ss_sales_price,ss_quantity]
CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name]
CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id]
CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk,ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy]
+ CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -30,17 +30,17 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #4
+ CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4
CometProject [d_date_sk,d_year,d_moy,d_qoy]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
- CometBroadcastExchange #5
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_store_id] #5
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6
+ CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
WholeStageCodegen (5)
HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt
index 9e72edbb4..feca96b18 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt
@@ -26,8 +26,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [ss_store_sk,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -35,11 +35,11 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
InputAdapter
BroadcastExchange #6
@@ -47,7 +47,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
BroadcastHashJoin [s_state,s_state]
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_county,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state]
InputAdapter
BroadcastExchange #7
@@ -64,16 +64,16 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_state,ss_net_profit]
+ CometHashAggregate [s_state,sum,ss_net_profit]
CometProject [ss_net_profit,s_state]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk]
CometProject [ss_net_profit,ss_sold_date_sk,s_state]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state]
+ CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
- CometBroadcastExchange #9
- CometFilter [s_store_sk]
+ CometBroadcastExchange [s_store_sk,s_state] #9
+ CometFilter [s_store_sk,s_state]
CometScan parquet spark_catalog.default.store [s_store_sk,s_state]
ReusedExchange [d_date_sk] #5
WholeStageCodegen (13)
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt
index bea1fd4a1..6cba2d0e6 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt
@@ -20,20 +20,20 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,d_date_sk,d_date,d_week_seq]
CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,hd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc,cd_demo_sk]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_sk,i_item_desc]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name]
- CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk,w_warehouse_sk,w_warehouse_name]
CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk]
- CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity]
- CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk]
+ CometBroadcastHashJoin [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
+ CometFilter [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
@@ -41,38 +41,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #4
- CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk]
+ CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4
+ CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk]
- CometBroadcastExchange #5
- CometFilter [w_warehouse_sk]
+ CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5
+ CometFilter [w_warehouse_sk,w_warehouse_name]
CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_desc] #6
+ CometFilter [i_item_sk,i_item_desc]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc]
- CometBroadcastExchange #7
+ CometBroadcastExchange [cd_demo_sk] #7
CometProject [cd_demo_sk]
- CometFilter [cd_marital_status,cd_demo_sk]
+ CometFilter [cd_demo_sk,cd_marital_status]
CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status]
- CometBroadcastExchange #8
+ CometBroadcastExchange [hd_demo_sk] #8
CometProject [hd_demo_sk]
- CometFilter [hd_buy_potential,hd_demo_sk]
+ CometFilter [hd_demo_sk,hd_buy_potential]
CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential]
- CometBroadcastExchange #9
+ CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9
CometProject [d_date_sk,d_date,d_week_seq]
- CometFilter [d_year,d_date_sk,d_week_seq,d_date]
+ CometFilter [d_date_sk,d_date,d_week_seq,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year]
- CometBroadcastExchange #10
- CometFilter [d_week_seq,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_week_seq] #10
+ CometFilter [d_date_sk,d_week_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq]
InputAdapter
BroadcastExchange #11
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
BroadcastExchange #12
@@ -90,5 +90,5 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt
index 283fe5bde..e5cf23f50 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt
@@ -12,25 +12,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #2
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #4
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #4
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #5
@@ -41,25 +41,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ss_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #7
- CometFilter [ss_customer_sk]
+ CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7
+ CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #8
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #9
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #9
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
BroadcastExchange #10
@@ -71,15 +71,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (4)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #12
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #12
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk,d_year] #4
@@ -92,15 +92,15 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid]
+ CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk,d_date_sk,d_year]
CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk]
- CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk]
- CometFilter [c_customer_sk,c_customer_id]
+ CometBroadcastHashJoin [c_customer_sk,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
+ CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name]
CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name]
- CometBroadcastExchange #15
- CometFilter [ws_bill_customer_sk]
+ CometBroadcastExchange [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] #15
+ CometFilter [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,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-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt
index fb78d64b1..2aecc9e98 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt
@@ -33,24 +33,24 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
- CometBroadcastExchange #6
+ CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
- CometBroadcastExchange #7
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #7
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -61,7 +61,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
WholeStageCodegen (10)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
@@ -75,10 +75,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -92,7 +92,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
WholeStageCodegen (15)
Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
@@ -106,10 +106,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -123,7 +123,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
InputAdapter
WholeStageCodegen (38)
@@ -156,21 +156,21 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [cs_item_sk,i_item_sk]
- CometFilter [cs_item_sk]
+ CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
SubqueryBroadcast [d_date_sk] #2
BroadcastExchange #17
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
- CometBroadcastExchange #18
- CometFilter [d_year,d_date_sk]
+ CometBroadcastExchange [d_date_sk,d_year] #18
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (23)
@@ -189,10 +189,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
@@ -214,10 +214,10 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
ColumnarToRow
InputAdapter
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_date_sk,d_year]
CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+ CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #2
ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt
index 752b8c854..34972d290 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt
@@ -24,12 +24,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit]
+ CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit]
CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk]
- CometBroadcastHashJoin [ss_store_sk,s_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,s_store_sk]
CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
- CometFilter [ss_store_sk]
+ CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk]
+ CometFilter [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -37,13 +37,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #6
+ CometBroadcastExchange [s_store_sk] #6
CometFilter [s_store_sk]
CometScan parquet spark_catalog.default.store [s_store_sk]
InputAdapter
@@ -55,12 +55,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (2)
ColumnarToRow
InputAdapter
- CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss]
+ CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss]
CometProject [sr_return_amt,sr_net_loss,s_store_sk]
- CometBroadcastHashJoin [sr_store_sk,s_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,s_store_sk]
CometProject [sr_store_sk,sr_return_amt,sr_net_loss]
- CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk]
- CometFilter [sr_store_sk]
+ CometBroadcastHashJoin [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk,d_date_sk]
+ CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -77,9 +77,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (5)
ColumnarToRow
InputAdapter
- CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
+ CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit]
CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit]
- CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -89,9 +89,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (7)
ColumnarToRow
InputAdapter
- CometHashAggregate [cr_return_amount,cr_net_loss]
+ CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss]
CometProject [cr_return_amount,cr_net_loss]
- CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk]
+ CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
@@ -104,16 +104,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (9)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit]
+ CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit]
CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
- CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,wp_web_page_sk]
CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_web_page_sk]
+ CometBroadcastHashJoin [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
- CometBroadcastExchange #13
+ CometBroadcastExchange [wp_web_page_sk] #13
CometFilter [wp_web_page_sk]
CometScan parquet spark_catalog.default.web_page [wp_web_page_sk]
InputAdapter
@@ -125,12 +125,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (10)
ColumnarToRow
InputAdapter
- CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss]
+ CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss]
CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk]
- CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wp_web_page_sk]
CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss]
- CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk]
- CometFilter [wr_web_page_sk]
+ CometBroadcastHashJoin [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk,d_date_sk]
+ CometFilter [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk]
ReusedSubquery [d_date_sk] #1
ReusedExchange [d_date_sk] #5
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt
index 49bd173f6..b8e6e0605 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt
@@ -27,14 +27,14 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_item_sk,ss_customer_sk]
+ CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #3
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [d_year,d_date_sk]
+ CometFilter [d_date_sk,d_year]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year]
InputAdapter
WholeStageCodegen (4)
@@ -45,7 +45,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number]
- CometFilter [sr_ticket_number,sr_item_sk]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (8)
ColumnarToRow
InputAdapter
- CometFilter [ws_item_sk,ws_bill_customer_sk]
+ CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -83,7 +83,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number]
- CometFilter [wr_order_number,wr_item_sk]
+ CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
@@ -109,7 +109,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cs_item_sk,cs_bill_customer_sk]
+ CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number]
- CometFilter [cr_order_number,cr_item_sk]
+ CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk,d_year] #3
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt
index 34e47dcba..0793067df 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt
@@ -39,7 +39,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk]
+ CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #5
@@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
InputAdapter
WholeStageCodegen (4)
@@ -58,7 +58,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss]
- CometFilter [sr_item_sk,sr_ticket_number]
+ CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -67,7 +67,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (6)
ColumnarToRow
InputAdapter
- CometFilter [s_store_sk]
+ CometFilter [s_store_sk,s_store_id]
CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id]
InputAdapter
BroadcastExchange #8
@@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [i_item_sk]
- CometFilter [i_current_price,i_item_sk]
+ CometFilter [i_item_sk,i_current_price]
CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price]
InputAdapter
BroadcastExchange #9
@@ -83,7 +83,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [p_promo_sk]
- CometFilter [p_channel_tv,p_promo_sk]
+ CometFilter [p_promo_sk,p_channel_tv]
CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv]
WholeStageCodegen (20)
HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty]
@@ -109,7 +109,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (11)
ColumnarToRow
InputAdapter
- CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk]
+ CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -121,7 +121,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss]
- CometFilter [cr_item_sk,cr_order_number]
+ CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (16)
ColumnarToRow
InputAdapter
- CometFilter [cp_catalog_page_sk]
+ CometFilter [cp_catalog_page_sk,cp_catalog_page_id]
CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
InputAdapter
ReusedExchange [i_item_sk] #8
@@ -160,7 +160,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (21)
ColumnarToRow
InputAdapter
- CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk]
+ CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk]
ReusedSubquery [d_date_sk] #1
InputAdapter
@@ -172,7 +172,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
ColumnarToRow
InputAdapter
CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss]
- CometFilter [wr_item_sk,wr_order_number]
+ CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk]
InputAdapter
ReusedExchange [d_date_sk] #5
@@ -181,7 +181,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
WholeStageCodegen (26)
ColumnarToRow
InputAdapter
- CometFilter [web_site_sk]
+ CometFilter [web_site_sk,web_site_id]
CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id]
InputAdapter
ReusedExchange [i_item_sk] #8
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt
index 0b7ad4726..5adee59be 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt
@@ -22,12 +22,12 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_category,i_class,ws_net_paid]
+ CometHashAggregate [i_category,i_class,sum,ws_net_paid]
CometProject [ws_net_paid,i_class,i_category]
- CometBroadcastHashJoin [ws_item_sk,i_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category]
CometProject [ws_item_sk,ws_net_paid]
- CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk]
- CometFilter [ws_item_sk]
+ CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk]
+ CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk]
CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -35,14 +35,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #5
+ CometBroadcastExchange [d_date_sk] #5
CometProject [d_date_sk]
- CometFilter [d_month_seq,d_date_sk]
+ CometFilter [d_date_sk,d_month_seq]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq]
- CometBroadcastExchange #6
- CometFilter [i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_class,i_category] #6
+ CometFilter [i_item_sk,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category]
WholeStageCodegen (5)
HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty]
diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt
index 6484c2dcb..64b162608 100644
--- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt
+++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt
@@ -17,12 +17,12 @@ WholeStageCodegen (5)
WholeStageCodegen (1)
ColumnarToRow
InputAdapter
- CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price]
+ CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price]
CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+ CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk]
CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastHashJoin [ss_item_sk,i_item_sk]
- CometFilter [ss_item_sk]
+ CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
+ CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk]
SubqueryBroadcast [d_date_sk] #1
BroadcastExchange #4
@@ -30,12 +30,12 @@ WholeStageCodegen (5)
ColumnarToRow
InputAdapter
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
- CometBroadcastExchange #5
- CometFilter [i_category,i_item_sk]
+ CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5
+ CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category]
- CometBroadcastExchange #6
+ CometBroadcastExchange [d_date_sk] #6
CometProject [d_date_sk]
- CometFilter [d_date,d_date_sk]
+ CometFilter [d_date_sk,d_date]
CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date]
diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala
index 9cc4e7f78..e657af9b9 100644
--- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala
+++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala
@@ -1438,7 +1438,7 @@ class CometExecSuite extends CometTestBase {
SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) {
withParquetTable((0 until 10).map(i => (i, 10 - i)), "t1") { // TODO: test nulls
val aggregateFunctions =
- List("COUNT(_1)", "MAX(_1)", "MIN(_1)") // TODO: Test all the aggregates
+ List("MAX(_1)", "MIN(_1)") // TODO: Test all the aggregates
aggregateFunctions.foreach { function =>
val queries = Seq(
diff --git a/spark/src/test/scala/org/apache/spark/sql/CometSQLQueryTestHelper.scala b/spark/src/test/scala/org/apache/spark/sql/CometSQLQueryTestHelper.scala
new file mode 100644
index 000000000..bf5ed4396
--- /dev/null
+++ b/spark/src/test/scala/org/apache/spark/sql/CometSQLQueryTestHelper.scala
@@ -0,0 +1,108 @@
+/*
+ * 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
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.{SparkException, SparkThrowable}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.execution.HiveResult.hiveResultString
+import org.apache.spark.sql.execution.SQLExecution
+import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase}
+import org.apache.spark.sql.types.StructType
+
+trait CometSQLQueryTestHelper {
+
+ private val notIncludedMsg = "[not included in comparison]"
+ private val clsName = this.getClass.getCanonicalName
+ protected val emptySchema: String = StructType(Seq.empty).catalogString
+
+ protected def replaceNotIncludedMsg(line: String): String = {
+ line
+ .replaceAll("#\\d+", "#x")
+ .replaceAll("plan_id=\\d+", "plan_id=x")
+ .replaceAll(s"Location.*$clsName/", s"Location $notIncludedMsg/{warehouse_dir}/")
+ .replaceAll(s"file:[^\\s,]*$clsName", s"file:$notIncludedMsg/{warehouse_dir}")
+ .replaceAll("Created By.*", s"Created By $notIncludedMsg")
+ .replaceAll("Created Time.*", s"Created Time $notIncludedMsg")
+ .replaceAll("Last Access.*", s"Last Access $notIncludedMsg")
+ .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg")
+ .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds
+ }
+
+ /** Executes a query and returns the result as (schema of the output, normalized output). */
+ protected def getNormalizedResult(session: SparkSession, sql: String): (String, Seq[String]) = {
+ // Returns true if the plan is supposed to be sorted.
+ def isSorted(plan: LogicalPlan): Boolean = plan match {
+ case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false
+ case _: DescribeCommandBase | _: DescribeColumnCommand | _: DescribeRelation |
+ _: DescribeColumn =>
+ true
+ case PhysicalOperation(_, _, Sort(_, true, _)) => true
+ case _ => plan.children.iterator.exists(isSorted)
+ }
+
+ val df = session.sql(sql)
+ val schema = df.schema.catalogString
+ // Get answer, but also get rid of the #1234 expression ids that show up in explain plans
+ val answer = SQLExecution.withNewExecutionId(df.queryExecution, Some(sql)) {
+ hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg)
+ }
+
+ // If the output is not pre-sorted, sort it.
+ if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted)
+ }
+
+ /**
+ * This method handles exceptions occurred during query execution as they may need special care
+ * to become comparable to the expected output.
+ *
+ * @param result
+ * a function that returns a pair of schema and output
+ */
+ protected def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = {
+ try {
+ result
+ } catch {
+ case e: SparkThrowable with Throwable if e.getErrorClass != null =>
+ (emptySchema, Seq(e.getClass.getName, e.getMessage))
+ case a: AnalysisException =>
+ // Do not output the logical plan tree which contains expression IDs.
+ // Also implement a crude way of masking expression IDs in the error message
+ // with a generic pattern "###".
+ val msg = a.getMessage
+ (emptySchema, Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x")))
+ case s: SparkException if s.getCause != null =>
+ // For a runtime exception, it is hard to match because its message contains
+ // information of stage, task ID, etc.
+ // To make result matching simpler, here we match the cause of the exception if it exists.
+ s.getCause match {
+ case e: SparkThrowable with Throwable if e.getErrorClass != null =>
+ (emptySchema, Seq(e.getClass.getName, e.getMessage))
+ case cause =>
+ (emptySchema, Seq(cause.getClass.getName, cause.getMessage))
+ }
+ case NonFatal(e) =>
+ // If there is an exception, put the exception class followed by the message.
+ (emptySchema, Seq(e.getClass.getName, e.getMessage))
+ }
+ }
+}
diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala
index 53186b131..6eeb7e334 100644
--- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala
@@ -27,10 +27,6 @@ import org.apache.comet.CometConf
class CometTPCDSQuerySuite
extends {
- override val excludedTpcdsQueries: Set[String] = Set()
-
- // This is private in `TPCDSBase` and `excludedTpcdsQueries` is private too.
- // So we cannot override `excludedTpcdsQueries` to exclude the queries.
val tpcdsAllQueries: Seq[String] = Seq(
"q1",
"q2",
@@ -112,7 +108,9 @@ class CometTPCDSQuerySuite
"q69",
"q70",
"q71",
- "q72",
+ // TODO: unknown failure (seems memory usage over Github action runner) in CI with q72 in
+ // https://github.com/apache/datafusion-comet/pull/613.
+ // "q72",
"q73",
"q74",
"q75",
@@ -141,11 +139,47 @@ class CometTPCDSQuerySuite
"q98",
"q99")
- // TODO: enable the 3 queries after fixing the issues #1358.
- override val tpcdsQueries: Seq[String] =
- tpcdsAllQueries.filterNot(excludedTpcdsQueries.contains)
+ val tpcdsAllQueriesV2_7_0: Seq[String] = Seq(
+ "q5a",
+ "q6",
+ "q10a",
+ "q11",
+ "q12",
+ "q14",
+ "q14a",
+ "q18a",
+ "q20",
+ "q22",
+ "q22a",
+ "q24",
+ "q27a",
+ "q34",
+ "q35",
+ "q35a",
+ "q36a",
+ "q47",
+ "q49",
+ "q51a",
+ "q57",
+ "q64",
+ "q67a",
+ "q70a",
+ // TODO: unknown failure (seems memory usage over Github action runner) in CI with q72-v2.7
+ // in https://github.com/apache/datafusion-comet/pull/613.
+ // "q72",
+ "q74",
+ "q75",
+ "q77a",
+ "q78",
+ "q80a",
+ "q86a",
+ "q98")
+
+ override val tpcdsQueries: Seq[String] = tpcdsAllQueries
+
+ override val tpcdsQueriesV2_7_0: Seq[String] = tpcdsAllQueriesV2_7_0
}
- with TPCDSQueryTestSuite
+ with CometTPCDSQueryTestSuite
with ShimCometTPCDSQuerySuite {
override def sparkConf: SparkConf = {
val conf = super.sparkConf
@@ -157,9 +191,11 @@ class CometTPCDSQuerySuite
conf.set(CometConf.COMET_EXEC_ENABLED.key, "true")
conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true")
conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true")
- conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "20g")
+ conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "15g")
+ conf.set(CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key, "true")
+ conf.set("spark.sql.adaptive.coalescePartitions.enabled", "true")
conf.set(MEMORY_OFFHEAP_ENABLED.key, "true")
- conf.set(MEMORY_OFFHEAP_SIZE.key, "20g")
+ conf.set(MEMORY_OFFHEAP_SIZE.key, "15g")
conf
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala
new file mode 100644
index 000000000..c2b853515
--- /dev/null
+++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueryTestSuite.scala
@@ -0,0 +1,230 @@
+/*
+ * 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
+
+import java.io.File
+import java.nio.file.{Files, Paths}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.catalyst.util.{fileToString, resourceToString, stringToFile}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.TestSparkSession
+
+/**
+ * Because we need to modify some methods of Spark `TPCDSQueryTestSuite` but they are private, we
+ * copy Spark `TPCDSQueryTestSuite`.
+ */
+class CometTPCDSQueryTestSuite extends QueryTest with TPCDSBase with CometSQLQueryTestHelper {
+
+ private val tpcdsDataPath = sys.env.get("SPARK_TPCDS_DATA")
+
+ private val regenGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1"
+
+ // To make output results deterministic
+ override protected def sparkConf: SparkConf = super.sparkConf
+ .set(SQLConf.SHUFFLE_PARTITIONS.key, "1")
+
+ protected override def createSparkSession: TestSparkSession = {
+ new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, sparkConf))
+ }
+
+ // We use SF=1 table data here, so we cannot use SF=100 stats
+ protected override val injectStats: Boolean = false
+
+ if (tpcdsDataPath.nonEmpty) {
+ val nonExistentTables = tableNames.filterNot { tableName =>
+ Files.exists(Paths.get(s"${tpcdsDataPath.get}/$tableName"))
+ }
+ if (nonExistentTables.nonEmpty) {
+ fail(
+ s"Non-existent TPCDS table paths found in ${tpcdsDataPath.get}: " +
+ nonExistentTables.mkString(", "))
+ }
+ }
+
+ protected val baseResourcePath: String = {
+ // use the same way as `SQLQueryTestSuite` to get the resource path
+ getWorkspaceFilePath(
+ "sql",
+ "core",
+ "src",
+ "test",
+ "resources",
+ "tpcds-query-results").toFile.getAbsolutePath
+ }
+
+ override def createTable(
+ spark: SparkSession,
+ tableName: String,
+ format: String = "parquet",
+ options: scala.Seq[String]): Unit = {
+ spark.sql(s"""
+ |CREATE TABLE `$tableName` (${tableColumns(tableName)})
+ |USING $format
+ |LOCATION '${tpcdsDataPath.get}/$tableName'
+ |${options.mkString("\n")}
+ """.stripMargin)
+ }
+
+ private def runQuery(query: String, goldenFile: File, conf: Map[String, String]): Unit = {
+ // This is `sortMergeJoinConf != conf` in Spark, i.e., it sorts results for other joins
+ // than sort merge join. But in some queries DataFusion sort returns correct results
+ // in terms of required sorting columns, but the results are not same as Spark in terms of
+ // order of irrelevant columns. So, we need to sort the results for all joins.
+ val shouldSortResults = true
+ withSQLConf(conf.toSeq: _*) {
+ try {
+ val (schema, output) = handleExceptions(getNormalizedResult(spark, query))
+ val queryString = query.trim
+ val outputString = output.mkString("\n").replaceAll("\\s+$", "")
+ if (regenGoldenFiles) {
+ val goldenOutput = {
+ s"-- Automatically generated by ${getClass.getSimpleName}\n\n" +
+ "-- !query schema\n" +
+ schema + "\n" +
+ "-- !query output\n" +
+ outputString +
+ "\n"
+ }
+ val parent = goldenFile.getParentFile
+ if (!parent.exists()) {
+ assert(parent.mkdirs(), "Could not create directory: " + parent)
+ }
+ stringToFile(goldenFile, goldenOutput)
+ }
+
+ // Read back the golden file.
+ val (expectedSchema, expectedOutput) = {
+ val goldenOutput = fileToString(goldenFile)
+ val segments = goldenOutput.split("-- !query.*\n")
+
+ // query has 3 segments, plus the header
+ assert(
+ segments.size == 3,
+ s"Expected 3 blocks in result file but got ${segments.size}. " +
+ "Try regenerate the result files.")
+
+ (segments(1).trim, segments(2).replaceAll("\\s+$", ""))
+ }
+
+ val notMatchedSchemaOutput = if (schema == emptySchema) {
+ // There might be exception. See `handleExceptions`.
+ s"Schema did not match\n$queryString\nOutput/Exception: $outputString"
+ } else {
+ s"Schema did not match\n$queryString"
+ }
+
+ assertResult(expectedSchema, notMatchedSchemaOutput) {
+ schema
+ }
+ if (shouldSortResults) {
+ val expectSorted = expectedOutput
+ .split("\n")
+ .sorted
+ .map(_.trim)
+ .mkString("\n")
+ .replaceAll("\\s+$", "")
+ val outputSorted = output.sorted.map(_.trim).mkString("\n").replaceAll("\\s+$", "")
+ assertResult(expectSorted, s"Result did not match\n$queryString") {
+ outputSorted
+ }
+ } else {
+ assertResult(expectedOutput, s"Result did not match\n$queryString") {
+ outputString
+ }
+ }
+ } catch {
+ case e: Throwable =>
+ val configs = conf.map { case (k, v) =>
+ s"$k=$v"
+ }
+ throw new Exception(s"${e.getMessage}\nError using configs:\n${configs.mkString("\n")}")
+ }
+ }
+ }
+
+ val sortMergeJoinConf: Map[String, String] = Map(
+ SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
+ SQLConf.PREFER_SORTMERGEJOIN.key -> "true")
+
+ val broadcastHashJoinConf: Map[String, String] = Map(
+ SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760")
+
+ val shuffledHashJoinConf: Map[String, String] = Map(
+ SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
+ "spark.sql.join.forceApplyShuffledHashJoin" -> "true")
+
+ val allJoinConfCombinations: Seq[Map[String, String]] =
+ Seq(sortMergeJoinConf, broadcastHashJoinConf, shuffledHashJoinConf)
+
+ val joinConfs: Seq[Map[String, String]] = if (regenGoldenFiles) {
+ require(
+ !sys.env.contains("SPARK_TPCDS_JOIN_CONF"),
+ "'SPARK_TPCDS_JOIN_CONF' cannot be set together with 'SPARK_GENERATE_GOLDEN_FILES'")
+ Seq(sortMergeJoinConf)
+ } else {
+ sys.env
+ .get("SPARK_TPCDS_JOIN_CONF")
+ .map { s =>
+ val p = new java.util.Properties()
+ p.load(new java.io.StringReader(s))
+ Seq(p.asScala.toMap)
+ }
+ .getOrElse(allJoinConfCombinations)
+ }
+
+ assert(joinConfs.nonEmpty)
+ joinConfs.foreach(conf =>
+ require(
+ allJoinConfCombinations.contains(conf),
+ s"Join configurations [$conf] should be one of $allJoinConfCombinations"))
+
+ if (tpcdsDataPath.nonEmpty) {
+ tpcdsQueries.foreach { name =>
+ val queryString = resourceToString(
+ s"tpcds/$name.sql",
+ classLoader = Thread.currentThread().getContextClassLoader)
+ test(name) {
+ val goldenFile = new File(s"$baseResourcePath/v1_4", s"$name.sql.out")
+ joinConfs.foreach { conf =>
+ System.gc() // Workaround for GitHub Actions memory limitation, see also SPARK-37368
+ runQuery(queryString, goldenFile, conf)
+ }
+ }
+ }
+
+ tpcdsQueriesV2_7_0.foreach { name =>
+ val queryString = resourceToString(
+ s"tpcds-v2.7.0/$name.sql",
+ classLoader = Thread.currentThread().getContextClassLoader)
+ test(s"$name-v2.7") {
+ val goldenFile = new File(s"$baseResourcePath/v2_7", s"$name.sql.out")
+ joinConfs.foreach { conf =>
+ System.gc() // SPARK-37368
+ runQuery(queryString, goldenFile, conf)
+ }
+ }
+ }
+ } else {
+ ignore("skipped because env `SPARK_TPCDS_DATA` is not set") {}
+ }
+}